diff --git a/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py b/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py index 244fe413ee5f8..12c8364720234 100644 --- a/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py +++ b/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py @@ -2,6 +2,7 @@ from dagster_buildkite.package_spec import PackageSpec from dagster_buildkite.python_version import AvailablePythonVersion +from dagster_buildkite.step_builder import BuildkiteQueue from dagster_buildkite.steps.packages import ( build_steps_from_package_specs, gcp_creds_extra_cmds, @@ -28,6 +29,7 @@ def build_dagster_oss_nightly_steps() -> List[BuildkiteStep]: unsupported_python_versions=[ AvailablePythonVersion.V3_12, ], + always_run_if=lambda: True, ), PackageSpec( "python_modules/libraries/dagster-k8s", @@ -41,6 +43,7 @@ def build_dagster_oss_nightly_steps() -> List[BuildkiteStep]: "nightly", ], pytest_extra_cmds=k8s_extra_cmds, + always_run_if=lambda: True, ), PackageSpec( "examples/experimental/dagster-dlift/kitchen-sink", @@ -52,6 +55,7 @@ def build_dagster_oss_nightly_steps() -> List[BuildkiteStep]: "KS_DBT_CLOUD_ACCESS_URL", "KS_DBT_CLOUD_DISCOVERY_API_URL", ], + always_run_if=lambda: True, ), PackageSpec( "examples/starlift-demo", @@ -63,6 +67,8 @@ def build_dagster_oss_nightly_steps() -> List[BuildkiteStep]: "KS_DBT_CLOUD_ACCESS_URL", "KS_DBT_CLOUD_DISCOVERY_API_URL", ], + queue=BuildkiteQueue.DOCKER, + always_run_if=lambda: True, ), PackageSpec( "integration_tests/test_suites/dagster-azure-live-tests", @@ -74,6 +80,7 @@ def build_dagster_oss_nightly_steps() -> List[BuildkiteStep]: "TEST_AZURE_STORAGE_ACCOUNT_ID", "TEST_AZURE_CONTAINER_ID", ], + always_run_if=lambda: True, ), ] ) diff --git a/docs/content/_navigation.json b/docs/content/_navigation.json index dc30c816dd041..045f10811069d 100644 --- a/docs/content/_navigation.json +++ b/docs/content/_navigation.json @@ -934,6 +934,10 @@ "title": "Airlift", "path": "/integrations/airlift", "children": [ + { + "title": "Mapping Airflow Concepts to Dagster", + "path": "/integrations/airlift/airflow-dagster-equivalents" + }, { "title": "Airflow Migration Tutorial", "path": "/integrations/airlift/tutorial/overview", @@ -985,6 +989,32 @@ { "title": "DAG-level migration", "path": "/integrations/airlift/full_dag" + }, + { + "title": "Migration with Airflow 1", + "path": "/integrations/airlift/airflow-1-migration" + }, + { + "title": "Operator Migration Reference", + "path": "/integrations/airlift/operator-migration/overview", + "children": [ + { + "title": "Migrating a BashOperator for dbt", + "path": "/integrations/airlift/operator-migration/bash-operator-dbt" + }, + { + "title": "Migrating a KubernetesPodOperator", + "path": "/integrations/airlift/operator-migration/kubernetes-pod-operator" + }, + { + "title": "Migrating a PythonOperator", + "path": "/integrations/airlift/operator-migration/python-operator" + }, + { + "title": "Migrating a BashOperator (general)", + "path": "/integrations/airlift/operator-migration/bash-operator-general" + } + ] } ] }, @@ -1094,7 +1124,13 @@ }, { "title": "Fivetran", - "path": "/integrations/fivetran" + "path": "/integrations/fivetran/fivetran", + "children": [ + { + "title": "Fivetran (Legacy)", + "path": "/integrations/fivetran/fivetran-legacy" + } + ] }, { "title": "Google BigQuery", diff --git a/docs/content/api/modules.json.gz b/docs/content/api/modules.json.gz index 889263d4901bf..babd11583e374 100644 Binary files a/docs/content/api/modules.json.gz and b/docs/content/api/modules.json.gz differ diff --git a/docs/content/api/searchindex.json.gz b/docs/content/api/searchindex.json.gz index 94f751562b023..5ca4d157c09eb 100644 Binary files a/docs/content/api/searchindex.json.gz and b/docs/content/api/searchindex.json.gz differ diff --git a/docs/content/api/sections.json.gz b/docs/content/api/sections.json.gz index 7695d0909a943..1254495534a4f 100644 Binary files a/docs/content/api/sections.json.gz and b/docs/content/api/sections.json.gz differ diff --git a/docs/content/concepts/assets/asset-checks.mdx b/docs/content/concepts/assets/asset-checks.mdx index 82037a5232753..c96a7fbb19aae 100644 --- a/docs/content/concepts/assets/asset-checks.mdx +++ b/docs/content/concepts/assets/asset-checks.mdx @@ -28,7 +28,7 @@ height={802} Using asset checks helps you: -- Create a single place in the Dagstser UI to learn about and investigate issues in your data pipeline +- Create a single place in the Dagster UI to learn about and investigate issues in your data pipeline - Ensure that data produced by your pipeline is timely and high quality by applying basic suites of tests to all assets - Identify problems in the source data that your pipeline depends on, such as checking data quality on source assets where dbt tests aren't possible - Communicate what the data produced by your pipeline is expected to look like (aka act as a data contract) @@ -89,7 +89,7 @@ The following table lists Dagster's built-in utility methods for creating asset | ----------------------------------------------------------- | --------------------------------------------------------------------------------------------------------- | | | Builds asset checks that pass if numeric metadata value falls within a particular range | | | Builds asset checks that pass if an asset's columns are the same, compared with its prior materialization | -| | Builds asset checks that pass if not too much time has elapsed since the latest time an asset was updated | +| | Builds asset checks that pass if they were updated within a given threshold | | | Builds asset checks that pass if an asset's most recent partition has been materialized before a deadline | ### Limitations diff --git a/docs/content/concepts/metadata-tags/kind-tags.mdx b/docs/content/concepts/metadata-tags/kind-tags.mdx index 1c9688bcf3523..3125a021b0c6e 100644 --- a/docs/content/concepts/metadata-tags/kind-tags.mdx +++ b/docs/content/concepts/metadata-tags/kind-tags.mdx @@ -137,6 +137,7 @@ Some kinds are given a branded icon in the UI. We currently support nearly 200 u | `huggingface` | | | `huggingfaceapi` | | | `iceberg` | | +| `icechunk` | | | `impala` | | | `instagram` | | | `ipynb` | | diff --git a/docs/content/integrations/airlift.mdx b/docs/content/integrations/airlift.mdx index a3b5e15e954d7..ff632972e869e 100644 --- a/docs/content/integrations/airlift.mdx +++ b/docs/content/integrations/airlift.mdx @@ -54,6 +54,12 @@ In this tutorial, we'll use `dagster-airlift` to observe DAGs from multiple Airf [Click here to get started](/integrations/airlift/federation-tutorial/overview). +## Airlift Operator Migration Reference + +In this reference, we'll explain how to migrate common Airflow operators to Dagster. + +[Click here to get started](/integrations/airlift/operator-migration/overview). + ## References @@ -69,4 +75,8 @@ In this tutorial, we'll use `dagster-airlift` to observe DAGs from multiple Airf title="Additional Airlift Functionality" href="/integrations/airlift/reference" > + diff --git a/docs/content/integrations/airlift/airflow-1-migration.mdx b/docs/content/integrations/airlift/airflow-1-migration.mdx new file mode 100644 index 0000000000000..b99b294ee7d0e --- /dev/null +++ b/docs/content/integrations/airlift/airflow-1-migration.mdx @@ -0,0 +1,278 @@ +# Airflow 1.X Migration Tutorial + +## Overview + +This guide covers using `dagster-airlift` to migrate an Airflow DAG to Dagster on `apache-airflow` below version 2. + +Many APIs within the `dagster-airlift` package make use of Airflow's stable REST API, which was added in Airflow 2.0. However, we still enable a migration process for Airflow 1.x users. + +This guide will cover the migration process using the same base example as the [tutorial](/integrations/airlift/tutorial/overview). + +We recommend following the tutorial in order to understand the concepts and steps involved in the migration process, and then using this guide to apply those steps to an Airflow 1.x environment. + +## Prerequisites + +Before continuining, you should have + +- Basic familiarity with Dagster concepts such as [assets](/concepts/assets/software-defined-assets) and [code locations](/concepts/code-locations). +- Skimmed the [Airlift migration tutorial](/integrations/airlift/tutorial/overview) to understand the general process of migration. + +### Setup + + + +If you previously ran the Airlift tutorial, you can follow along by doing the following: + +- clear `tutorial_example/dagster_defs/definitions.py`, and mark all tasks as unproxied in the proxied state YAML file. + + + + +Start by following the [setup](/integrations/airlift/tutorial/setup) step of the migration tutorial, and we'll diverge from there. + +With Airflow 1.x, we won't [peer](/integrations/airlift/tutorial/peer) or [observe](/integrations/airlift/tutorial/observe) Airflow DAGs first - we'll immediately skip to the migration step and proxy execution to Dagster. + +### Scaffolding proxied state + +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: + +```yaml file=../../airlift-migration-tutorial/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml +tasks: + - id: load_raw_customers + proxied: False + - id: build_dbt_models + proxied: False + - id: export_customers + proxied: False +``` + +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: + +```python file=../../airlift-migration-tutorial/tutorial_example/snippets/dags_truncated.py +# Dags file can be found at tutorial_example/airflow_dags/dags.py +from pathlib import Path + +from airflow import DAG +from dagster_airlift.in_airflow import proxying_to_dagster +from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml + +dag = DAG("rebuild_customers_list", ...) + +... + +# Set this to True to begin the proxying process +PROXYING = False + +if PROXYING: + proxying_to_dagster( + global_vars=globals(), + proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), + ) +``` + +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.) + +

+ + + +

+ +### Migrating `build_dbt_models` + +We'll now create Dagster assets that correspond to each Airflow task. First, since Dagster provides out of the box integration with dbt, we'll use some utilities from `dagster-dbt` to create assets for the `build_dbt_models` task. In our `tutorial_example/dagster_defs/definitions.py` file: + +```python file=../../airlift-migration-tutorial/snippets/airflow_1_dbt.py startafter=start_dbt_project_assets endbefore=end_dbt_project_assets +import os +from pathlib import Path + +from dagster import AssetExecutionContext +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() +``` + +We used the decorator to create assets from the dbt project. The `manifest` argument points to the manifest file generated by dbt, and the `project` argument points to the dbt project directory. The resource is a wrapper around the dbt CLI, which we use to execute dbt commands. + +Now, we'll mark our `dbt_project_assets` as being mapped from Airflow: + +```python file=../../airlift-migration-tutorial/snippets/airflow_1_dbt.py startafter=start_mapping endbefore=end_mapping +from dagster_airlift.core import assets_with_task_mappings + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "build_dbt_models": + # load rich set of assets from dbt project + [dbt_project_assets], + }, +) +``` + +The `assets_with_task_mappings` function adds some metadata to each passed-in asset which, over the wire in Airflow, we'll use to determine which assets to execute in Dagster. + +We'll provide the mapped assets to a `Definitions` object in our `tutorial_example/dagster_defs/definitions.py` file: + +```python file=../../airlift-migration-tutorial/snippets/airflow_1_dbt.py startafter=start_defs endbefore=end_defs +from dagster import Definitions + +defs = Definitions( + assets=mapped_assets, resources={"dbt": DbtCliResource(project_dir=dbt_project_path())} +) +``` + +Note how this differs from the original migration tutorial; we're not using `build_defs_from_airflow_instance`, which relies on the REST API. + +Finally, we'll mark the `build_dbt_models` task as proxied in the proxied state YAML file: + +```yaml file=../../airlift-migration-tutorial/tutorial_example/snippets/dbt_proxied.yaml +tasks: + - id: load_raw_customers + proxied: False + - id: build_dbt_models + proxied: True + - id: export_customers + proxied: False +``` + +**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: + +

+ + + +

+ +### Completed code + +Migrating the other tasks should follow the same pattern as in the [migration tutorial](/integrations/airlift/tutorial/migrate#migrating-the-remaining-custom-operators). When you're done, your code should look like this: + +```python file=../../airlift-migration-tutorial/snippets/airflow_1_migrated.py +import os +from pathlib import Path + +from dagster import ( + AssetExecutionContext, + AssetsDefinition, + AssetSpec, + DailyPartitionsDefinition, + Definitions, + multi_asset, +) +from dagster._time import get_current_datetime_midnight +from dagster_airlift.core import assets_with_task_mappings +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +# Code also invoked from Airflow +from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv +from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb + +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +def airflow_dags_path() -> Path: + return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" + + +def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: + @multi_asset(name=f"load_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + load_csv_to_duckdb(args) + + return _multi_asset + + +def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: + @multi_asset(name=f"export_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + export_duckdb_to_csv(args) + + return _multi_asset + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "load_raw_customers": [ + load_csv_to_duckdb_asset( + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF), + LoadCsvToDuckDbArgs( + table_name="raw_customers", + csv_path=airflow_dags_path() / "raw_customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + names=["id", "first_name", "last_name"], + duckdb_schema="raw_data", + duckdb_database_name="jaffle_shop", + ), + ) + ], + "build_dbt_models": + # load rich set of assets from dbt project + [dbt_project_assets], + "export_customers": [ + export_duckdb_to_csv_defs( + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF), + ExportDuckDbToCsvArgs( + table_name="customers", + csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + duckdb_database_name="jaffle_shop", + ), + ) + ], + }, +) + + +defs = Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, +) +``` + +### Conclusion + +To recap, we've covered the process of migrating an Airflow 1.x DAG to Dagster using `dagster-airlift`. We've made clearer what functionality works wth Airflow < 2.0, and what does not. We've shown how to create Dagster assets that correspond to Airflow tasks, and how to mark those tasks as proxied in the proxied state YAML file. diff --git a/docs/content/integrations/airlift/airflow-dagster-equivalents.mdx b/docs/content/integrations/airlift/airflow-dagster-equivalents.mdx new file mode 100644 index 0000000000000..3a89b8f529eee --- /dev/null +++ b/docs/content/integrations/airlift/airflow-dagster-equivalents.mdx @@ -0,0 +1,617 @@ +# Airflow to Dagster concepts + +In this page, we'll explain how concepts map between Airflow and Dagster. + +### Airflow Tasks + +An Airflow Task is a single computation within a DAG, which executes an `execute` method of its given Operator. Here's an example of an Airflow Task using the `PythonOperator` to upload a file to S3, as well as the taskflow-based `@task` decorator (which achieves the same thing): + +```python file=/integrations/airlift/equivalents/airflow_task.py +from pathlib import Path + +import boto3 +from airflow.decorators import task +from airflow.operators.python import PythonOperator + + +def write_file_to_s3(path: Path) -> None: + boto3.client("s3").upload_file(str(path), "bucket", path.name) + + +def _write_customers_data(): + write_file_to_s3(Path("path/to/customers.csv")) + + +# Defining a task using the PythonOperator syntax +PythonOperator( + python_callable=_write_customers_data, task_id="write_customers_data", dag=... +) + + +# Defining a task using the task decorator +@task(task_id="write_customers_data") +def write_customers_data(): + write_file_to_s3(Path("path/to/customers.csv")) +``` + +Dagster in comparison uses software-defined assets to represent what a task produces. You define the data asset you want to exist, and then define a computation for _how_ to create that asset. If you're familiar with Airflow's Datasets, you can think of a software-defined asset as defining a task and a dataset at once. Here's an example of a Dagster asset that uploads a file to S3: + +```python file=/integrations/airlift/equivalents/task_equiv_asset.py +from pathlib import Path + +import boto3 + +from dagster import asset + + +def write_file_to_s3(path: Path) -> None: + boto3.client("s3").upload_file(str(path), "bucket", path.name) + + +# We define in terms of the "physical" asset - the uploaded file +@asset +def customers_data(): + write_file_to_s3(Path("path/to/customers.csv")) +``` + +Learn more about Dagster's asset-oriented approach in our [software-defined assets reference](/concepts/assets/software-defined-assets). + +### Airflow Operators + +An Airflow Operator is a "building block" which can be used to define tasks from. Here's an example of a `PythonOperator` subclass which takes a file as an input and uploads it to S3: + +```python file=/integrations/airlift/equivalents/custom_operator.py startafter=start_custom_op endbefore=end_custom_op +from pathlib import Path + +import boto3 +from airflow.operators.python import PythonOperator + + +class UploadToS3Operator(PythonOperator): + def __init__(self, path: str, *args, **kwargs) -> None: + super().__init__( + python_callable=self.upload_to_s3, op_args=[path], *args, **kwargs + ) + + def upload_to_s3(self, path: str) -> None: + boto3.client("s3").upload_file( + Filepath=path, Bucket="my_bucket", Key=Path(path).name + ) +``` + +Then, you would invoke this operator to create a task like so: + +```python file=/integrations/airlift/equivalents/custom_operator.py startafter=start_task endbefore=end_task +task = UploadToS3Operator(task_id="write_customers_data", path="path/to/customers.csv") +``` + +The Dagster equivalent would be to create a factory method that defines an asset for a given parameter. + +```python file=/integrations/airlift/equivalents/factory_asset.py startafter=start_factory endbefore=end_factory +import boto3 + +from dagster import asset + + +def build_s3_asset(path: str): + @asset(key=path) + def _s3_file(): + boto3.client("s3").upload_file(path, "my-bucket", path) + + return _s3_file +``` + +Then, you would invoke this factory method to create an asset like so: + +```python file=/integrations/airlift/equivalents/factory_asset.py startafter=start_usage endbefore=end_usage +customers_data = build_s3_asset("path/to/customers.csv") +``` + +### Airflow DAGs & Schedules + +An Airflow DAG is a collection of tasks with dependencies between them, and some scheduling information. Here's an example of an Airflow DAG which runs on a daily schedule: + +```python file=/integrations/airlift/equivalents/airflow_dag.py +from datetime import datetime, timedelta + +from airflow import DAG +from airflow.decorators import task + +dag = DAG( + "simple_dag", + start_date=datetime(2024, 1, 1), + schedule_interval="@daily", +) + + +@task(task_id="write_customers_data", dag=dag) +def write_customers_data(): ... +``` + +You can define a schedule on an arbitrary set of assets in Dagster. Here's an example of a Dagster Asset that runs on a daily schedule: + +```python file=/integrations/airlift/equivalents/scheduled_asset.py +from dagster import ScheduleDefinition, asset, define_asset_job + + +@asset +def customers_data(): ... + + +ScheduleDefinition( + "daily_schedule", + cron_schedule="0 0 * * *", + target=customers_data, +) +``` + +### Airflow DagBags + +An Airflow DagBag is a collection of dags parsed from a particular file tree. The closest equivalent in Dagster is a code location, where you can set up collections of assets, schedules, sensors, etc. See our [code locations guide](/concepts/code-locations) for more information. + +### Airflow data interval, Logical Date, and Execution Date + +In Airflow, data interval is the range of data being processed by the dag, with logical date and execution date being synonymous with the "start" of the data interval. + +```python file=/integrations/airlift/equivalents/airflow_data_interval.py +import boto3 +from airflow.decorators import task + + +@task(task_id="write_customers_data") +def write_partitioned_customers_data(context): + prefix = context["data_interval_start"] + # or + prefix = context["logical_date"] + # or + prefix = context["execution_date"] + + # write data to S3 with the prefix + boto3.client("s3").upload_file( + "path/to/customers.csv", f"bucket/{prefix}/customers.csv" + ) +``` + +The equivalent concept in Dagster is partitions - where you can define a partitioning scheme to represent the data being processed by your computations. + +```python file=/integrations/airlift/equivalents/dagster_partition.py +import boto3 + +from dagster import AssetExecutionContext, DailyPartitionsDefinition, asset + + +@asset(partitions_def=DailyPartitionsDefinition(...)) +def customers_data(context: AssetExecutionContext): + prefix = context.partition_key + boto3.client("s3").upload_file( + "path/to/customers.csv", f"bucket/{prefix}/customers.csv" + ) +``` + +To learn more, see our [partitions guide](/concepts/partitions-schedules-sensors/partitions). + +### Airflow Sensors & Triggers + +Both Airflow and Dagster have the concept of sensors, but they function very differently. In Airflow, sensors, are used to wait for a certain condition to be met before proceeding with the execution of a task. Let's take the example scenario of waiting for new files to exist before proceeding. + +```python file=/integrations/airlift/equivalents/airflow_sensor.py +from datetime import datetime + +from airflow import DAG +from airflow.sensors.filesystem import FileSensor + +dag = DAG("file_sensor_example", start_date=datetime(2024, 1, 1)) + +wait_for_file = FileSensor( + task_id="wait_for_new_customer_files", + filepath="/path/to/customer_files/*.csv", + poke_interval=60, # Check every minute + timeout=3600, # Timeout after 1 hour + mode="poke", + dag=dag, +) +``` + +Triggers in Airflow are an event-based way to accomplish a similar task: waiting for some condition to be true. They run asynchronously in a separate process. + +Dagster sensors combine the best of both worlds of Airflow sensors and triggers, while also providing additional capabilities. Here's an example of a Dagster sensor that will kick off computation of an asset whenever a new file is added to a directory: + +```python file=/integrations/airlift/equivalents/dagster_sensor.py +import json +from pathlib import Path + +from dagster import RunRequest, SensorEvaluationContext, SkipReason, asset, sensor + + +@asset +def uploaded_customers_data(): + pass + + +# Implementing the FileSensor from Airflow directly in Dagster +@sensor(target=uploaded_customers_data) +def wait_for_new_files(context: SensorEvaluationContext): + seen_files: list = json.loads(context.cursor) if context.cursor else [] + should_trigger = False + for file in Path("path/to/customer_files").iterdir(): + if file.name not in seen_files: + seen_files.append(file.name) + should_trigger = True + yield RunRequest() if should_trigger else SkipReason("No new files") +``` + +Key features of Dagster sensors: + +- In Dagster, sensors run _continuously_ and _independently_. This means that they don't use up a task slot in the scheduler, and can poll much more frequently than Airflow sensors. +- Sensors in Dagster can track state between evaluations using cursors. +- Sensors in Dagster can do more than trigger downstream computations - they can also be used to set run tags, trigger external APIs, and more. They are ultimately arbitrary Python functions that can do anything you want (although we generally recommend keeping them lightweight, since they are designed to poll often and not run on heavy infrastructure). + +#### A note on Airflow's ExternalTaskSensor + +Dagster can handle execution of downstreams automatically using [Declarative Automation](/concepts/automation/declarative-automation) - so you don't need to worry about cross-dag dependencies like you do in Airflow. So, for example, if you were previously using the ExternalTaskSensor in Airflow to wait for a task in another DAG to complete, you don't need to do that in Dagster. Instead, you would just define a dependency between those assets in the asset graph. + +### Airflow Hooks & Connections + +Hooks in Airflow are used to interact with external systems, and Connections are used to authenticate to those systems. For example, you would first set up a connection to AWS using an environment variable: + +```bash +export AIRFLOW_CONN_AWS_DEFAULT='aws://YOUR_AWS_ACCESS_KEY_ID:YOUR_AWS_SECRET_ACCESS_KEY@/?region_name=us-east-1' +``` + +Here's an example of using an Airflow Hook with a set-up Airflow connection to interact with S3: + +```python file=/integrations/airlift/equivalents/airflow_hook.py startafter=start_ex endbefore=end_ex +from datetime import datetime, timedelta + +from airflow import DAG +from airflow.operators.python import PythonOperator +from airflow.providers.amazon.aws.hooks.s3 import S3Hook + + +def upload_customers_data() -> None: + S3Hook(aws_conn_id="aws_default").load_file( + filename="path/to/customers.csv", + key="customers.csv", + bucket_name="my-cool-bucket", + replace=True, + ) + + +s3_task = PythonOperator( + task_id="s3_operations", + python_callable=upload_customers_data, +) +``` + +In Dagster, you interact with external systems using resources, and _configure_ those resources with specific connection information. Here's an example of a Dagster Resource to interact with S3, set up with S3 credentials: + +```python file=/integrations/airlift/equivalents/dagster_resource.py +import os +from pathlib import Path + +from dagster_aws.s3 import S3Resource + +from dagster import Definitions, asset + + +@asset +def customers_s3(s3: S3Resource): + local_file_data = Path("path/to/customers_data.csv").read_bytes() + s3_client = s3.get_client() + + s3_client.put_object( + Bucket="my-cool-bucket", + Key="customers.csv", + Body=local_file_data, + ) + + +defs = Definitions( + assets=[customers_s3], + resources={ + "s3": S3Resource(aws_access_key_id=os.environ["DEFAULT_AWS_ACCESS_KEY_ID"]) + }, +) +``` + +### Airflow XComs + +Airflow XComs are used to pass data between tasks. Here's an example of an Airflow XCom: + +```python file=/integrations/airlift/equivalents/airflow_xcom.py +import boto3 +import pandas as pd +from airflow.operators.python import PythonOperator + + +def upload_customers_data(**context): + raw_customers_data = pd.read_csv("path/to/customers.csv") + avg_revenue = raw_customers_data["revenue"].mean() + task_instance = context["task_instance"] + task_instance.xcom_push(key="avg_revenue", value=avg_revenue) + boto3.client("s3").upload_file("path/to/customers.csv", "bucket/customers.csv") + + +PythonOperator( + task_id="generate_stats", + python_callable=upload_customers_data, + provide_context=True, +) +``` + +In Dagster, you can automatically pass small amounts of data using `Asset metadata`. Here's an example of a Dagster Asset that passes data between tasks: + +```python file=/integrations/airlift/equivalents/dagster_asset.py +import boto3 +import pandas as pd + +from dagster import FloatMetadataValue, MaterializeResult, asset + + +@asset +def customers_data_s3(): + raw_customers_data = pd.read_csv("path/to/customers.csv") + avg_revenue = raw_customers_data["revenue"].mean() + boto3.client("s3").upload_file("path/to/customers.csv", "bucket/customers.csv") + return MaterializeResult(metadata={"avg_revenue": FloatMetadataValue(avg_revenue)}) +``` + +For larger amounts of data, you can use Dagster's [IOManager](/concepts/io-management/io-managers) to manage the data flow between tasks. + +### Airflow Templates & Macros + +Airflow allows templating variables into your DAGs using Jinja. Dagster doesn't have a direct jinja templating feature, instead you're encouraged to use Python functions and interact with Dagster's richly typed API to pass information. The context information available to each asset's execution is on the object. + +### Airflow Executors + +Airflow Executors are used to determine where your tasks are executed. You can use the equivalent Dagster executors concept for this purpose. Learn more about [Dagster executors](/deployment/executors). + +### Airflow Pools + +Airflow Pools allow users to limit the number of concurrent tasks that can be run. Dagster provides concurrency at various levels of the stack to limit the number of computations that can run at a given time, and also limit the number of runs that are in progress at a given time. View our full concurrency guide [here](/guides/limiting-concurrency-in-data-pipelines). + +### Airflow Task Groups + +Airflow task groups allow you to organize tasks into hierarchical groups within the Airflow UI for a particular DAG. Dagster has _global_ asset groups which can be applied to any asset. Learn more about [asset groups](/concepts/assets/software-defined-assets#assigning-assets-to-groups). + +## Cheatsheet + +Here's a cheatsheet for Airflow users migrating to Dagster: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ Airflow concept + + Dagster concept + Notes
Directed Acyclic Graphs (DAG) + Jobs +
Tasks & Datasets + Assets + + Dagster assets combine the concepts of tasks and datasets into a single + abstraction that can be used to define both the computation and the data + produced by that computation. They include support for things like{" "} + + partitioning + +
Connections/Hooks + + + Dagster's resource system allows you to define and configure connections + to external systems. Dagster+ also has a dedicated secrets management + system. +
Variables + + + Dagster's rich config system allows you to define configuration + schematics in code, and pass configuration to computations at runtime. +
DagBags + Code locations + + Multiple isolated code locations with different system and Python + dependencies can exist within the same Dagster deployment. +
DAG runsAsset Runs
+ depends_on_past + + + + An asset can{" "} + + depend on earlier partitions of itself + + . When this is the case, + backfills + and + Declarative Automation + will only materialize later partitions after earlier partitions have + completed. +
Executors + Executors +
Instances + Instances +
OperatorsAsset Factories + Dagster uses normal Python functions instead of framework-specific + operator classes. To define multiple assets from a shared + implementation, you can use a factory function. +
Pools & Task Concurrency + + Asset & Run Concurrency + +
Plugins/Providers + Integrations +
Schedulers + Schedules +
Sensors & Triggers + Sensors, + + Declarative Automation + + + Dagster Sensors provide the best of both worlds of Airflow's sensors and + triggers, while also providing additional functionality. Declarative + automation makes the need for cross-dag dependency sensors like in + Airflow unnecessary. +
SubDAGs/TaskGroups + + + Dagster provides rich, searchable metadata and{" "} + tagging support beyond what's + offered by Airflow. +
XComs + Runtime metadata + I/O managers + + For small data, you can use Dagster's rich metadata abstractions to make + data available between assets, and have it show up in the UI. For larger + datasets, I/O managers are more powerful than XComs and allow the + passing large datasets between jobs. +
diff --git a/docs/content/integrations/airlift/operator-migration/bash-operator-dbt.mdx b/docs/content/integrations/airlift/operator-migration/bash-operator-dbt.mdx new file mode 100644 index 0000000000000..30ab8f9ef47e2 --- /dev/null +++ b/docs/content/integrations/airlift/operator-migration/bash-operator-dbt.mdx @@ -0,0 +1,53 @@ +# Operator migration guides: Migrating usage of `BashOperator` for `dbt` + +In this page, we'll explain migrating an Airflow `BashOperator` that runs a `dbt` command to Dagster. + +### Background + +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. + +```python file=/integrations/airlift/operator_migration/bash_operator_dbt.py +from airflow.operators.bash import BashOperator + +run_dbt_model = BashOperator(task_id="build_dbt_models", bash_command="dbt run") +``` + +### 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: + +```python file=/integrations/airlift/operator_migration/using_dbt_assets.py +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +from dagster import AssetExecutionContext + +project = DbtProject(project_dir="path/to/dbt_project") + + +@dbt_assets(manifest=project.manifest_path) +def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["run"], context=context).stream() +``` + +### 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/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/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/dbt/quickstart) and [reference](/integrations/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. The [dagster-airlift migration guide](/integrations/airlift/tutorial/overview) details this process. diff --git a/docs/content/integrations/airlift/operator-migration/bash-operator-general.mdx b/docs/content/integrations/airlift/operator-migration/bash-operator-general.mdx new file mode 100644 index 0000000000000..f37b13c9e2602 --- /dev/null +++ b/docs/content/integrations/airlift/operator-migration/bash-operator-general.mdx @@ -0,0 +1,77 @@ +# Operator migration guides: Migrating generalized usage of BashOperator + +In this page, we'll explain migrating an Airflow `BashOperator` to Dagster. + + + If using the `BashOperator` to execute dbt commands, see the [dbt migration + guide](/integrations/airlift/operator-migration/bash-operator-dbt). + + +### Background + +The Airflow `BashOperator` is a common operator used to execute bash commands as part of a data pipeline. + +```python file=/integrations/airlift/operator_migration/bash_operator_general.py +from airflow.operators.bash import BashOperator + +execute_script = BashOperator( + task_id="execute_script", + bash_command="python /path/to/script.py", +) +``` + +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 . + +```python file=/integrations/airlift/operator_migration/using_pipes_subprocess.py +from dagster import AssetExecutionContext, PipesSubprocessClient, asset + + +@asset +def script_result(context: AssetExecutionContext): + return ( + PipesSubprocessClient() + .run(context=context, command="python /path/to/script.py") + .get_results() + ) +``` + +### Step 3: Using `dagster-airlift` to proxy execution + +Finally, you can use `dagster-airlift` to proxy the execution of the original task to Dagster. The [dagster-airlift migration guide](/integrations/airlift/tutorial/overview) details this process. + +### 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](/concepts/dagster-pipes/subprocess) for more information. + +#### Running a dbt command + +We have a whole guide for switching from the `BashOperator` to the `dbt` integration in Dagster. See the [dbt migration guide](/integrations/airlift/operator-migration/bash-operator-dbt) for more information. + +#### 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/content/integrations/airlift/operator-migration/kubernetes-pod-operator.mdx b/docs/content/integrations/airlift/operator-migration/kubernetes-pod-operator.mdx new file mode 100644 index 0000000000000..90590c4050a43 --- /dev/null +++ b/docs/content/integrations/airlift/operator-migration/kubernetes-pod-operator.mdx @@ -0,0 +1,159 @@ +# Operator migration guides: Migrating usage of KubernetesPodOperator + +In this page, we'll explain migrating an Airflow `KubernetesPodOperator` to Dagster. + +### Background + +The KubernetesPodOperator in Apache Airflow enables users to execute containerized tasks within Kubernetes pods as part of their data pipelines. + +```python file=/integrations/airlift/operator_migration/kubernetes_pod_operator.py +from airflow.providers.cncf.kubernetes.operators.pod import KubernetesPodOperator + +k8s_hello_world = KubernetesPodOperator( + task_id="hello_world_task", + name="hello-world-pod", + image="bash:latest", + cmds=["bash", "-cx"], + arguments=['echo "Hello World!"'], +) +``` + +### Dagster equivalent + +The Dagster equivalent is to use the to execute a task within a Kubernetes pod. + +```python file=/integrations/airlift/operator_migration/using_k8s_pipes.py +from dagster_k8s import PipesK8sClient + +from dagster import AssetExecutionContext, asset + +container_cfg = { + "name": "hello-world-pod", + "image": "bash:latest", + "command": ["bash", "-cx"], + "args": ['echo "Hello World!"'], +} + + +@asset +def execute_hello_world_task(context: AssetExecutionContext): + return ( + PipesK8sClient() + .run( + context=context, + base_pod_meta={"name": "hello-world-pod"}, + base_pod_spec={"containers": [container_cfg]}, + ) + .get_results() + ) +``` + +### 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: + +```python file=/integrations/airlift/operator_migration/k8s_eks_fake_example.py startafter=start_client endbefore=end_client +from dagster_k8s import PipesK8sClient + +eks_client = PipesK8sClient( + # The client will have automatic access to all + # environment variables in the execution context. + env={**AWS_CREDENTIALS, "AWS_REGION": "us-west-2"}, + kubeconfig_file="path/to/kubeconfig", + kube_context="my-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](/integrations/airlift/operator-migration/kubernetes-pod-operator#parameter-comparison) section of this doc, you'll find a detailed comparison describing how to map the KubernetesPodOperator parameters to the PipesK8sClient parameters. + +```python file=/integrations/airlift/operator_migration/k8s_eks_fake_example.py startafter=start_asset endbefore=end_asset +from dagster import AssetExecutionContext, asset + +container_cfg = { + "name": "hello-world-pod", + "image": "bash:latest", + "command": ["bash", "-cx"], + "args": ['echo "Hello World!"'], +} + + +@asset +def execute_hello_world_task(context: AssetExecutionContext): + return eks_client.run( + context=context, + base_pod_meta={"name": "hello-world-pod"}, + base_pod_spec={"containers": [container_cfg]}, + ).get_results() +``` + +This is just a snippet of what the PipesK8sClient can do. Take a look at our full guide on the [dagster-k8s PipesK8sClient](/concepts/dagster-pipes/kubernetes) 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. The [dagster-airlift migration guide](/integrations/airlift/tutorial/overview) details this process. + +### 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/content/integrations/airlift/operator-migration/overview.mdx b/docs/content/integrations/airlift/operator-migration/overview.mdx new file mode 100644 index 0000000000000..2829d757d3e4f --- /dev/null +++ b/docs/content/integrations/airlift/operator-migration/overview.mdx @@ -0,0 +1,24 @@ +# Operator Migration Reference + +This page contains a collection of reference materials for migrating usage of common Airflow operator types to Dagster. + +## References + + + + + + + diff --git a/docs/content/integrations/airlift/operator-migration/python-operator.mdx b/docs/content/integrations/airlift/operator-migration/python-operator.mdx new file mode 100644 index 0000000000000..d11b37b45b2e4 --- /dev/null +++ b/docs/content/integrations/airlift/operator-migration/python-operator.mdx @@ -0,0 +1,124 @@ +# Migrating the PythonOperator to Dagster + +In this page, we'll explain migrating an Airflow `PythonOperator` to Dagster. + +### Background + +In Airflow, the `PythonOperator` runs arbitrary python functions. For example, you might have a task that runs function `write_to_db`, which combs a directory for files, and writes each one to a db table. + +```python file=/integrations/airlift/operator_migration/python_operator.py startafter=start_op endbefore=end_op +from airflow.operators.python import PythonOperator + + +def write_to_db() -> None: + for raw_file in RAW_DATA_DIR.iterdir(): + df = contents_as_df(raw_file) + upload_to_db(df) + + +PythonOperator(python_callable=write_to_db, task_id="db_upload", dag=...) +``` + +### Dagster equivalent + +The Dagster equivalent is instead to construct a or -decorated function, which materializes assets corresponding to what your python function is doing. + +```python file=/integrations/airlift/operator_migration/pyop_multi_asset_complete.py startafter=start_asset endbefore=end_asset +from dagster import asset + + +@asset(key=TABLE_URI) +def write_to_db() -> None: + for raw_file in RAW_DATA_DIR.iterdir(): + df = contents_as_df(raw_file) + upload_to_db(df) +``` + +### 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: + +```python file=/integrations/airlift/operator_migration/python_operator.py startafter=start_op endbefore=end_op +from airflow.operators.python import PythonOperator + + +def write_to_db() -> None: + for raw_file in RAW_DATA_DIR.iterdir(): + df = contents_as_df(raw_file) + upload_to_db(df) + + +PythonOperator(python_callable=write_to_db, task_id="db_upload", dag=...) +``` + +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: + +```python file=/integrations/airlift/operator_migration/python_operator.py startafter=start_shared endbefore=end_shared +from airflow.operators.python import PythonOperator +from shared_module import write_to_db + +PythonOperator(python_callable=write_to_db, task_id="db_upload", dag=...) +``` + +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. + +```python file=/integrations/airlift/operator_migration/pyop_asset_shared.py +# start_asset +# This would be the python code living in a shared module. +from shared_module import my_shared_python_callable + +from dagster import asset + + +@asset +def my_shared_asset(): + return my_shared_python_callable() + + +# end_asset +``` + +### Step 3: Using `dagster-airlift` to proxy execution + +Finally, you can use `dagster-airlift` to proxy the execution of the original task to Dagster. The [dagster-airlift migration guide](/integrations/airlift/tutorial/overview) details this process. diff --git a/docs/content/integrations/airlift/reference.mdx b/docs/content/integrations/airlift/reference.mdx index f1adda58eb2ca..24cc925b99427 100644 --- a/docs/content/integrations/airlift/reference.mdx +++ b/docs/content/integrations/airlift/reference.mdx @@ -12,6 +12,14 @@ description: "dagster-airlift is a toolkit for observing and migrating Airflow D - [Dealing with changing Airflow](#dealing-with-changing-airflow) - [Automating changes to code locations](#automating-changes-to-code-locations) +### 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. diff --git a/docs/content/integrations/embedded-elt/dlt.mdx b/docs/content/integrations/embedded-elt/dlt.mdx index cec177f548147..fb9f62066834b 100644 --- a/docs/content/integrations/embedded-elt/dlt.mdx +++ b/docs/content/integrations/embedded-elt/dlt.mdx @@ -245,6 +245,7 @@ from dagster_embedded_elt.dlt import ( DagsterDltTranslator, dlt_assets, ) +from dlt.extract.resource import DltResource from dagster import AssetExecutionContext, AssetKey @@ -257,11 +258,11 @@ def example_dlt_source(): class CustomDagsterDltTranslator(DagsterDltTranslator): - def get_asset_key(self, resource: DagsterDltResource) -> AssetKey: + def get_asset_key(self, resource: DltResource) -> AssetKey: """Overrides asset key to be the dlt resource name.""" return AssetKey(f"{resource.name}") - def get_deps_asset_keys(self, resource: DagsterDltResource) -> Iterable[AssetKey]: + def get_deps_asset_keys(self, resource: DltResource) -> Iterable[AssetKey]: """Overrides upstream asset key to be a single source asset.""" return [AssetKey("common_upstream_dlt_dependency")] diff --git a/docs/content/integrations/fivetran.mdx b/docs/content/integrations/fivetran/fivetran-legacy.mdx similarity index 99% rename from docs/content/integrations/fivetran.mdx rename to docs/content/integrations/fivetran/fivetran-legacy.mdx index a9c9b942bb177..0451e58a648cd 100644 --- a/docs/content/integrations/fivetran.mdx +++ b/docs/content/integrations/fivetran/fivetran-legacy.mdx @@ -1,9 +1,9 @@ --- -title: "Fivetran & Dagster | Dagster Docs" +title: "Fivetran & Dagster | Dagster Docs (Legacy)" description: Integrate your Fivetran connectors into Dagster. --- -# Fivetran & Dagster +# Fivetran & Dagster (Legacy) Dagster can orchestrate your Fivetran connectors, making it easy to chain a Fivetran sync with upstream or downstream steps in your workflow. diff --git a/docs/content/integrations/fivetran/fivetran.mdx b/docs/content/integrations/fivetran/fivetran.mdx new file mode 100644 index 0000000000000..cd33c2331e76e --- /dev/null +++ b/docs/content/integrations/fivetran/fivetran.mdx @@ -0,0 +1,184 @@ +--- +title: "Using Dagster with Fivetran" +description: Represent your Fivetran connectors in Dagster +--- + +# Using Dagster with Fivetran + +This guide provides instructions for using Dagster with Fivetran using the `dagster-fivetran` library. Your Fivetran connector tables can be represented as assets in the Dagster asset graph, allowing you to track lineage and dependencies between Fivetran assets and data assets you are already modeling in Dagster. You can also use Dagster to orchestrate Fivetran connectors, allowing you to trigger syncs for these on a cadence or based on upstream data changes. + +## What you'll learn + +- How to represent Fivetran assets in the Dagster asset graph, including lineage to other Dagster assets. +- How to customize asset definition metadata for these Fivetran assets. +- How to materialize Fivetran connector tables from Dagster. +- How to customize how Fivetran connector tables are materialized. + +
+ Prerequisites + +- The `dagster` and `dagster-fivetran` libraries installed in your environment +- Familiarity with asset definitions and the Dagster asset graph +- Familiarity with Dagster resources +- Familiarity with Fivetran concepts, like connectors and connector tables +- A Fivetran workspace +- A Fivetran API key and API secret. For more information, see [Getting Started](https://fivetran.com/docs/rest-api/getting-started) in the Fivetran REST API documentation. + +
+ +## Represent Fivetran assets in the asset graph + +To load Fivetran assets into the Dagster asset graph, you must first construct a resource, which allows Dagster to communicate with your Fivetran workspace. You'll need to supply your account ID, API key and API secret. See [Getting Started](https://fivetran.com/docs/rest-api/getting-started) in the Fivetran REST API documentation for more information on how to create your API key and API secret. + +Dagster can automatically load all connector tables from your Fivetran workspace as asset specs. Call the function, which returns list of s representing your Fivetran assets. You can then include these asset specs in your object: + +```python file=/integrations/fivetran/representing_fivetran_assets.py +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}) +``` + +### Sync and materialize Fivetran assets + +You can use Dagster to sync Fivetran connectors and materialize Fivetran connector tables. You can use the factory to create all assets definitions for your Fivetran workspace. + +```python file=/integrations/fivetran/sync_and_materialize_fivetran_assets.py +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"), +) + +all_fivetran_assets = build_fivetran_assets_definitions(workspace=fivetran_workspace) + +defs = dg.Definitions( + assets=all_fivetran_assets, + resources={"fivetran": fivetran_workspace}, +) +``` + +### Customize the materialization of Fivetran assets + +If you want to customize the sync of your connectors, you can use the decorator to do so. This allows you to execute custom code before and after the call to the fivetran sync. + +```python file=/integrations/fivetran/customize_fivetran_asset_defs.py +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 +): + # Do something before the materialization... + yield from fivetran.sync_and_poll(context=context) + # Do something after the materialization... + + +defs = dg.Definitions( + assets=[fivetran_connector_assets], + resources={"fivetran": fivetran_workspace}, +) +``` + +### Customize asset definition metadata for Fivetran assets + +By default, Dagster will generate asset specs for each Fivetran asset and populate default metadata. You can further customize asset properties by passing an instance of the custom to the function. + +```python file=/integrations/fivetran/customize_fivetran_translator_asset_spec.py +from dagster_fivetran import ( + DagsterFivetranTranslator, + FivetranConnectorTableProps, + 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"), +) + + +# A translator class lets us customize properties of the built +# Fivetran assets, such as the owners or asset key +class MyCustomFivetranTranslator(DagsterFivetranTranslator): + def get_asset_spec(self, props: FivetranConnectorTableProps) -> dg.AssetSpec: + # We create the default asset spec using super() + default_spec = super().get_asset_spec(props) + # We customize the metadata and asset key prefix for all assets + return default_spec.replace_attributes( + key=default_spec.key.with_prefix("prefix"), + ).merge_attributes(metadata={"custom": "metadata"}) + + +fivetran_specs = load_fivetran_asset_specs( + fivetran_workspace, dagster_fivetran_translator=MyCustomFivetranTranslator() +) + +defs = dg.Definitions(assets=fivetran_specs, resources={"fivetran": fivetran_workspace}) +``` + +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 Fivetran assets from multiple workspaces + +Definitions from multiple Fivetran workspaces can be combined by instantiating multiple resources and merging their specs. This lets you view all your Fivetran assets in a single asset graph: + +```python file=/integrations/fivetran/multiple_fivetran_workspaces.py +from dagster_fivetran import FivetranWorkspace, load_fivetran_asset_specs + +import dagster as dg + +sales_fivetran_workspace = FivetranWorkspace( + account_id=dg.EnvVar("FIVETRAN_SALES_ACCOUNT_ID"), + api_key=dg.EnvVar("FIVETRAN_SALES_API_KEY"), + api_secret=dg.EnvVar("FIVETRAN_SALES_API_SECRET"), +) +marketing_fivetran_workspace = FivetranWorkspace( + account_id=dg.EnvVar("FIVETRAN_MARKETING_ACCOUNT_ID"), + api_key=dg.EnvVar("FIVETRAN_MARKETING_API_KEY"), + api_secret=dg.EnvVar("FIVETRAN_MARKETING_API_SECRET"), +) + +sales_fivetran_specs = load_fivetran_asset_specs(sales_fivetran_workspace) +marketing_fivetran_specs = load_fivetran_asset_specs(marketing_fivetran_workspace) + +# Merge the specs into a single set of definitions +defs = dg.Definitions( + assets=[*sales_fivetran_specs, *marketing_fivetran_specs], + resources={ + "marketing_fivetran": marketing_fivetran_workspace, + "sales_fivetran": sales_fivetran_workspace, + }, +) +``` diff --git a/docs/dagster-university/yarn.lock b/docs/dagster-university/yarn.lock index aeed20689d3a4..249e336397af5 100644 --- a/docs/dagster-university/yarn.lock +++ b/docs/dagster-university/yarn.lock @@ -3467,11 +3467,11 @@ __metadata: linkType: hard "nanoid@npm:^3.3.6": - version: 3.3.6 - resolution: "nanoid@npm:3.3.6" + version: 3.3.8 + resolution: "nanoid@npm:3.3.8" bin: nanoid: bin/nanoid.cjs - checksum: 10/67235c39d1bc05851383dadde5cf77ae1c90c2a1d189e845c7f20f646f0488d875ad5f5226bbba072a88cebbb085a3f784a6673117daf785bdf614a852550362 + checksum: 10/2d1766606cf0d6f47b6f0fdab91761bb81609b2e3d367027aff45e6ee7006f660fb7e7781f4a34799fe6734f1268eeed2e37a5fdee809ade0c2d4eb11b0f9c40 languageName: node linkType: hard diff --git a/docs/next/public/images/concepts/metadata-tags/kinds/icons/tool-icechunk-color.svg b/docs/next/public/images/concepts/metadata-tags/kinds/icons/tool-icechunk-color.svg new file mode 100644 index 0000000000000..3de7fb2c0b315 --- /dev/null +++ b/docs/next/public/images/concepts/metadata-tags/kinds/icons/tool-icechunk-color.svg @@ -0,0 +1,1088 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/next/public/objects.inv b/docs/next/public/objects.inv index dcf6a751e8f2e..fd50d97032019 100644 Binary files a/docs/next/public/objects.inv and b/docs/next/public/objects.inv differ diff --git a/docs/sphinx/sections/api/apidocs/libraries/dagster-airbyte.rst b/docs/sphinx/sections/api/apidocs/libraries/dagster-airbyte.rst index 7e0024acb3f87..b9a4dc1299f85 100644 --- a/docs/sphinx/sections/api/apidocs/libraries/dagster-airbyte.rst +++ b/docs/sphinx/sections/api/apidocs/libraries/dagster-airbyte.rst @@ -8,23 +8,24 @@ For more information on getting started, see the `Airbyte integration guide Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +# end_dbt_project_assets + +# start_mapping +from dagster_airlift.core import assets_with_task_mappings + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "build_dbt_models": + # load rich set of assets from dbt project + [dbt_project_assets], + }, +) +# end_mapping + +# start_defs +from dagster import Definitions + +defs = Definitions( + assets=mapped_assets, resources={"dbt": DbtCliResource(project_dir=dbt_project_path())} +) +# end_defs diff --git a/examples/airlift-migration-tutorial/snippets/airflow_1_migrated.py b/examples/airlift-migration-tutorial/snippets/airflow_1_migrated.py new file mode 100644 index 0000000000000..a60482bd31bd5 --- /dev/null +++ b/examples/airlift-migration-tutorial/snippets/airflow_1_migrated.py @@ -0,0 +1,95 @@ +import os +from pathlib import Path + +from dagster import ( + AssetExecutionContext, + AssetsDefinition, + AssetSpec, + DailyPartitionsDefinition, + Definitions, + multi_asset, +) +from dagster._time import get_current_datetime_midnight +from dagster_airlift.core import assets_with_task_mappings +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +# Code also invoked from Airflow +from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv +from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb + +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +def airflow_dags_path() -> Path: + return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" + + +def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: + @multi_asset(name=f"load_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + load_csv_to_duckdb(args) + + return _multi_asset + + +def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: + @multi_asset(name=f"export_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + export_duckdb_to_csv(args) + + return _multi_asset + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "load_raw_customers": [ + load_csv_to_duckdb_asset( + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF), + LoadCsvToDuckDbArgs( + table_name="raw_customers", + csv_path=airflow_dags_path() / "raw_customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + names=["id", "first_name", "last_name"], + duckdb_schema="raw_data", + duckdb_database_name="jaffle_shop", + ), + ) + ], + "build_dbt_models": + # load rich set of assets from dbt project + [dbt_project_assets], + "export_customers": [ + export_duckdb_to_csv_defs( + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF), + ExportDuckDbToCsvArgs( + table_name="customers", + csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + duckdb_database_name="jaffle_shop", + ), + ) + ], + }, +) + + +defs = Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, +) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_dag.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_dag.py new file mode 100644 index 0000000000000..e5553c57ae82e --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_dag.py @@ -0,0 +1,14 @@ +from datetime import datetime, timedelta + +from airflow import DAG +from airflow.decorators import task + +dag = DAG( + "simple_dag", + start_date=datetime(2024, 1, 1), + schedule_interval="@daily", +) + + +@task(task_id="write_customers_data", dag=dag) +def write_customers_data(): ... diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_data_interval.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_data_interval.py new file mode 100644 index 0000000000000..7cb0e6fa974ba --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_data_interval.py @@ -0,0 +1,16 @@ +import boto3 +from airflow.decorators import task + + +@task(task_id="write_customers_data") +def write_partitioned_customers_data(context): + prefix = context["data_interval_start"] + # or + prefix = context["logical_date"] + # or + prefix = context["execution_date"] + + # write data to S3 with the prefix + boto3.client("s3").upload_file( + "path/to/customers.csv", f"bucket/{prefix}/customers.csv" + ) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_hook.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_hook.py new file mode 100644 index 0000000000000..bee33967dffac --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_hook.py @@ -0,0 +1,23 @@ +# type: ignore +# start_ex +from datetime import datetime, timedelta + +from airflow import DAG +from airflow.operators.python import PythonOperator +from airflow.providers.amazon.aws.hooks.s3 import S3Hook + + +def upload_customers_data() -> None: + S3Hook(aws_conn_id="aws_default").load_file( + filename="path/to/customers.csv", + key="customers.csv", + bucket_name="my-cool-bucket", + replace=True, + ) + + +s3_task = PythonOperator( + task_id="s3_operations", + python_callable=upload_customers_data, +) +# end_ex diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_sensor.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_sensor.py new file mode 100644 index 0000000000000..7b7a4872f5aa4 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_sensor.py @@ -0,0 +1,15 @@ +from datetime import datetime + +from airflow import DAG +from airflow.sensors.filesystem import FileSensor + +dag = DAG("file_sensor_example", start_date=datetime(2024, 1, 1)) + +wait_for_file = FileSensor( + task_id="wait_for_new_customer_files", + filepath="/path/to/customer_files/*.csv", + poke_interval=60, # Check every minute + timeout=3600, # Timeout after 1 hour + mode="poke", + dag=dag, +) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_task.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_task.py new file mode 100644 index 0000000000000..a8f8b8941e19c --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_task.py @@ -0,0 +1,25 @@ +from pathlib import Path + +import boto3 +from airflow.decorators import task +from airflow.operators.python import PythonOperator + + +def write_file_to_s3(path: Path) -> None: + boto3.client("s3").upload_file(str(path), "bucket", path.name) + + +def _write_customers_data(): + write_file_to_s3(Path("path/to/customers.csv")) + + +# Defining a task using the PythonOperator syntax +PythonOperator( + python_callable=_write_customers_data, task_id="write_customers_data", dag=... +) + + +# Defining a task using the task decorator +@task(task_id="write_customers_data") +def write_customers_data(): + write_file_to_s3(Path("path/to/customers.csv")) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_xcom.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_xcom.py new file mode 100644 index 0000000000000..712cdcfaf0ed0 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/airflow_xcom.py @@ -0,0 +1,18 @@ +import boto3 +import pandas as pd +from airflow.operators.python import PythonOperator + + +def upload_customers_data(**context): + raw_customers_data = pd.read_csv("path/to/customers.csv") + avg_revenue = raw_customers_data["revenue"].mean() + task_instance = context["task_instance"] + task_instance.xcom_push(key="avg_revenue", value=avg_revenue) + boto3.client("s3").upload_file("path/to/customers.csv", "bucket/customers.csv") + + +PythonOperator( + task_id="generate_stats", + python_callable=upload_customers_data, + provide_context=True, +) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/custom_operator.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/custom_operator.py new file mode 100644 index 0000000000000..b114238c40e37 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/custom_operator.py @@ -0,0 +1,24 @@ +# start_custom_op +from pathlib import Path + +import boto3 +from airflow.operators.python import PythonOperator + + +class UploadToS3Operator(PythonOperator): + def __init__(self, path: str, *args, **kwargs) -> None: + super().__init__( + python_callable=self.upload_to_s3, op_args=[path], *args, **kwargs + ) + + def upload_to_s3(self, path: str) -> None: + boto3.client("s3").upload_file( + Filepath=path, Bucket="my_bucket", Key=Path(path).name + ) + + +# end_custom_op + +# start_task +task = UploadToS3Operator(task_id="write_customers_data", path="path/to/customers.csv") +# end_task diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_asset.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_asset.py new file mode 100644 index 0000000000000..73a644b139a04 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_asset.py @@ -0,0 +1,12 @@ +import boto3 +import pandas as pd + +from dagster import FloatMetadataValue, MaterializeResult, asset + + +@asset +def customers_data_s3(): + raw_customers_data = pd.read_csv("path/to/customers.csv") + avg_revenue = raw_customers_data["revenue"].mean() + boto3.client("s3").upload_file("path/to/customers.csv", "bucket/customers.csv") + return MaterializeResult(metadata={"avg_revenue": FloatMetadataValue(avg_revenue)}) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_partition.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_partition.py new file mode 100644 index 0000000000000..ebcbb2dda5dba --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_partition.py @@ -0,0 +1,11 @@ +import boto3 + +from dagster import AssetExecutionContext, DailyPartitionsDefinition, asset + + +@asset(partitions_def=DailyPartitionsDefinition(...)) +def customers_data(context: AssetExecutionContext): + prefix = context.partition_key + boto3.client("s3").upload_file( + "path/to/customers.csv", f"bucket/{prefix}/customers.csv" + ) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_resource.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_resource.py new file mode 100644 index 0000000000000..9d0ec547a3d12 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_resource.py @@ -0,0 +1,26 @@ +import os +from pathlib import Path + +from dagster_aws.s3 import S3Resource + +from dagster import Definitions, asset + + +@asset +def customers_s3(s3: S3Resource): + local_file_data = Path("path/to/customers_data.csv").read_bytes() + s3_client = s3.get_client() + + s3_client.put_object( + Bucket="my-cool-bucket", + Key="customers.csv", + Body=local_file_data, + ) + + +defs = Definitions( + assets=[customers_s3], + resources={ + "s3": S3Resource(aws_access_key_id=os.environ["DEFAULT_AWS_ACCESS_KEY_ID"]) + }, +) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_sensor.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_sensor.py new file mode 100644 index 0000000000000..1d698b405c6c1 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/dagster_sensor.py @@ -0,0 +1,21 @@ +import json +from pathlib import Path + +from dagster import RunRequest, SensorEvaluationContext, SkipReason, asset, sensor + + +@asset +def uploaded_customers_data(): + pass + + +# Implementing the FileSensor from Airflow directly in Dagster +@sensor(target=uploaded_customers_data) +def wait_for_new_files(context: SensorEvaluationContext): + seen_files: list = json.loads(context.cursor) if context.cursor else [] + should_trigger = False + for file in Path("path/to/customer_files").iterdir(): + if file.name not in seen_files: + seen_files.append(file.name) + should_trigger = True + yield RunRequest() if should_trigger else SkipReason("No new files") diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/factory_asset.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/factory_asset.py new file mode 100644 index 0000000000000..2dc779ec52aa8 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/factory_asset.py @@ -0,0 +1,19 @@ +# start_factory +import boto3 + +from dagster import asset + + +def build_s3_asset(path: str): + @asset(key=path) + def _s3_file(): + boto3.client("s3").upload_file(path, "my-bucket", path) + + return _s3_file + + +# end_factory + +# start_usage +customers_data = build_s3_asset("path/to/customers.csv") +# end_usage diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/scheduled_asset.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/scheduled_asset.py new file mode 100644 index 0000000000000..0f81181b8d78d --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/scheduled_asset.py @@ -0,0 +1,12 @@ +from dagster import ScheduleDefinition, asset, define_asset_job + + +@asset +def customers_data(): ... + + +ScheduleDefinition( + "daily_schedule", + cron_schedule="0 0 * * *", + target=customers_data, +) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/task_equiv_asset.py b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/task_equiv_asset.py new file mode 100644 index 0000000000000..cd368682109c2 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/equivalents/task_equiv_asset.py @@ -0,0 +1,15 @@ +from pathlib import Path + +import boto3 + +from dagster import asset + + +def write_file_to_s3(path: Path) -> None: + boto3.client("s3").upload_file(str(path), "bucket", path.name) + + +# We define in terms of the "physical" asset - the uploaded file +@asset +def customers_data(): + write_file_to_s3(Path("path/to/customers.csv")) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/bash_operator_dbt.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/bash_operator_dbt.py new file mode 100644 index 0000000000000..7f500a680aebd --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/bash_operator_dbt.py @@ -0,0 +1,3 @@ +from airflow.operators.bash import BashOperator + +run_dbt_model = BashOperator(task_id="build_dbt_models", bash_command="dbt run") diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/bash_operator_general.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/bash_operator_general.py new file mode 100644 index 0000000000000..1a45e386c8567 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/bash_operator_general.py @@ -0,0 +1,6 @@ +from airflow.operators.bash import BashOperator + +execute_script = BashOperator( + task_id="execute_script", + bash_command="python /path/to/script.py", +) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/k8s_eks_fake_example.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/k8s_eks_fake_example.py new file mode 100644 index 0000000000000..8ff590044138f --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/k8s_eks_fake_example.py @@ -0,0 +1,35 @@ +AWS_CREDENTIALS = {} + +# start_client +from dagster_k8s import PipesK8sClient + +eks_client = PipesK8sClient( + # The client will have automatic access to all + # environment variables in the execution context. + env={**AWS_CREDENTIALS, "AWS_REGION": "us-west-2"}, + kubeconfig_file="path/to/kubeconfig", + kube_context="my-eks-cluster", +) +# end_client + +# start_asset +from dagster import AssetExecutionContext, asset + +container_cfg = { + "name": "hello-world-pod", + "image": "bash:latest", + "command": ["bash", "-cx"], + "args": ['echo "Hello World!"'], +} + + +@asset +def execute_hello_world_task(context: AssetExecutionContext): + return eks_client.run( + context=context, + base_pod_meta={"name": "hello-world-pod"}, + base_pod_spec={"containers": [container_cfg]}, + ).get_results() + + +# end_asset diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/kubernetes_pod_operator.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/kubernetes_pod_operator.py new file mode 100644 index 0000000000000..bde2a3e5b1a86 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/kubernetes_pod_operator.py @@ -0,0 +1,9 @@ +from airflow.providers.cncf.kubernetes.operators.pod import KubernetesPodOperator + +k8s_hello_world = KubernetesPodOperator( + task_id="hello_world_task", + name="hello-world-pod", + image="bash:latest", + cmds=["bash", "-cx"], + arguments=['echo "Hello World!"'], +) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/pyop_asset_shared.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/pyop_asset_shared.py new file mode 100644 index 0000000000000..7cac49711e3b1 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/pyop_asset_shared.py @@ -0,0 +1,14 @@ +# type: ignore +# start_asset +# This would be the python code living in a shared module. +from shared_module import my_shared_python_callable + +from dagster import asset + + +@asset +def my_shared_asset(): + return my_shared_python_callable() + + +# end_asset diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/pyop_multi_asset_complete.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/pyop_multi_asset_complete.py new file mode 100644 index 0000000000000..91a1768e2dd13 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/pyop_multi_asset_complete.py @@ -0,0 +1,27 @@ +from pathlib import Path +from typing import Any + +RAW_DATA_DIR = Path("path") +TABLE_URI = "blah" + + +def contents_as_df(path: Path) -> Any: + pass + + +def upload_to_db(df): + pass + + +# start_asset +from dagster import asset + + +@asset(key=TABLE_URI) +def write_to_db() -> None: + for raw_file in RAW_DATA_DIR.iterdir(): + df = contents_as_df(raw_file) + upload_to_db(df) + + +# end_asset diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/python_operator.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/python_operator.py new file mode 100644 index 0000000000000..014e23a4bb7fc --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/python_operator.py @@ -0,0 +1,35 @@ +# type: ignore +from pathlib import Path +from typing import Any + +RAW_DATA_DIR = Path("path") + + +def contents_as_df(path: Path) -> Any: + pass + + +def upload_to_db(df: Any): + pass + + +# start_op +from airflow.operators.python import PythonOperator + + +def write_to_db() -> None: + for raw_file in RAW_DATA_DIR.iterdir(): + df = contents_as_df(raw_file) + upload_to_db(df) + + +PythonOperator(python_callable=write_to_db, task_id="db_upload", dag=...) + +# end_op + +# start_shared +from airflow.operators.python import PythonOperator +from shared_module import write_to_db + +PythonOperator(python_callable=write_to_db, task_id="db_upload", dag=...) +# end_shared diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/shared.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/shared.py new file mode 100644 index 0000000000000..83089bcd9c742 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/shared.py @@ -0,0 +1,2 @@ +def my_shared_python_callable(): + pass diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_dbt_assets.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_dbt_assets.py new file mode 100644 index 0000000000000..ea9937de0d4db --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_dbt_assets.py @@ -0,0 +1,10 @@ +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +from dagster import AssetExecutionContext + +project = DbtProject(project_dir="path/to/dbt_project") + + +@dbt_assets(manifest=project.manifest_path) +def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["run"], context=context).stream() diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_k8s_pipes.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_k8s_pipes.py new file mode 100644 index 0000000000000..986125ea5daa4 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_k8s_pipes.py @@ -0,0 +1,23 @@ +from dagster_k8s import PipesK8sClient + +from dagster import AssetExecutionContext, asset + +container_cfg = { + "name": "hello-world-pod", + "image": "bash:latest", + "command": ["bash", "-cx"], + "args": ['echo "Hello World!"'], +} + + +@asset +def execute_hello_world_task(context: AssetExecutionContext): + return ( + PipesK8sClient() + .run( + context=context, + base_pod_meta={"name": "hello-world-pod"}, + base_pod_spec={"containers": [container_cfg]}, + ) + .get_results() + ) diff --git a/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_pipes_subprocess.py b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_pipes_subprocess.py new file mode 100644 index 0000000000000..29418f3aa6b38 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/airlift/operator_migration/using_pipes_subprocess.py @@ -0,0 +1,10 @@ +from dagster import AssetExecutionContext, PipesSubprocessClient, asset + + +@asset +def script_result(context: AssetExecutionContext): + return ( + PipesSubprocessClient() + .run(context=context, command="python /path/to/script.py") + .get_results() + ) diff --git a/examples/docs_snippets/docs_snippets/integrations/embedded_elt/dlt_dagster_translator.py b/examples/docs_snippets/docs_snippets/integrations/embedded_elt/dlt_dagster_translator.py index ffa4e1ae40482..8f6c19b6691de 100644 --- a/examples/docs_snippets/docs_snippets/integrations/embedded_elt/dlt_dagster_translator.py +++ b/examples/docs_snippets/docs_snippets/integrations/embedded_elt/dlt_dagster_translator.py @@ -6,6 +6,7 @@ DagsterDltTranslator, dlt_assets, ) +from dlt.extract.resource import DltResource from dagster import AssetExecutionContext, AssetKey @@ -18,11 +19,11 @@ def example_resource(): ... class CustomDagsterDltTranslator(DagsterDltTranslator): - def get_asset_key(self, resource: DagsterDltResource) -> AssetKey: + def get_asset_key(self, resource: DltResource) -> AssetKey: """Overrides asset key to be the dlt resource name.""" return AssetKey(f"{resource.name}") - def get_deps_asset_keys(self, resource: DagsterDltResource) -> Iterable[AssetKey]: + def get_deps_asset_keys(self, resource: DltResource) -> Iterable[AssetKey]: """Overrides upstream asset key to be a single source asset.""" return [AssetKey("common_upstream_dlt_dependency")] diff --git a/examples/docs_snippets/docs_snippets/integrations/fivetran/customize_fivetran_asset_defs.py b/examples/docs_snippets/docs_snippets/integrations/fivetran/customize_fivetran_asset_defs.py new file mode 100644 index 0000000000000..b195ed0c134c3 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/fivetran/customize_fivetran_asset_defs.py @@ -0,0 +1,29 @@ +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 +): + # Do something before the materialization... + yield from fivetran.sync_and_poll(context=context) + # Do something after the materialization... + + +defs = dg.Definitions( + assets=[fivetran_connector_assets], + resources={"fivetran": fivetran_workspace}, +) diff --git a/examples/docs_snippets/docs_snippets/integrations/fivetran/customize_fivetran_translator_asset_spec.py b/examples/docs_snippets/docs_snippets/integrations/fivetran/customize_fivetran_translator_asset_spec.py new file mode 100644 index 0000000000000..8debde35e6eba --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/fivetran/customize_fivetran_translator_asset_spec.py @@ -0,0 +1,33 @@ +from dagster_fivetran import ( + DagsterFivetranTranslator, + FivetranConnectorTableProps, + 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"), +) + + +# A translator class lets us customize properties of the built +# Fivetran assets, such as the owners or asset key +class MyCustomFivetranTranslator(DagsterFivetranTranslator): + def get_asset_spec(self, props: FivetranConnectorTableProps) -> dg.AssetSpec: + # We create the default asset spec using super() + default_spec = super().get_asset_spec(props) + # We customize the metadata and asset key prefix for all assets + return default_spec.replace_attributes( + key=default_spec.key.with_prefix("prefix"), + ).merge_attributes(metadata={"custom": "metadata"}) + + +fivetran_specs = load_fivetran_asset_specs( + fivetran_workspace, dagster_fivetran_translator=MyCustomFivetranTranslator() +) + +defs = dg.Definitions(assets=fivetran_specs, resources={"fivetran": fivetran_workspace}) diff --git a/examples/docs_snippets/docs_snippets/integrations/fivetran/multiple_fivetran_workspaces.py b/examples/docs_snippets/docs_snippets/integrations/fivetran/multiple_fivetran_workspaces.py new file mode 100644 index 0000000000000..d28203bf3bcdf --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/fivetran/multiple_fivetran_workspaces.py @@ -0,0 +1,26 @@ +from dagster_fivetran import FivetranWorkspace, load_fivetran_asset_specs + +import dagster as dg + +sales_fivetran_workspace = FivetranWorkspace( + account_id=dg.EnvVar("FIVETRAN_SALES_ACCOUNT_ID"), + api_key=dg.EnvVar("FIVETRAN_SALES_API_KEY"), + api_secret=dg.EnvVar("FIVETRAN_SALES_API_SECRET"), +) +marketing_fivetran_workspace = FivetranWorkspace( + account_id=dg.EnvVar("FIVETRAN_MARKETING_ACCOUNT_ID"), + api_key=dg.EnvVar("FIVETRAN_MARKETING_API_KEY"), + api_secret=dg.EnvVar("FIVETRAN_MARKETING_API_SECRET"), +) + +sales_fivetran_specs = load_fivetran_asset_specs(sales_fivetran_workspace) +marketing_fivetran_specs = load_fivetran_asset_specs(marketing_fivetran_workspace) + +# Merge the specs into a single set of definitions +defs = dg.Definitions( + assets=[*sales_fivetran_specs, *marketing_fivetran_specs], + resources={ + "marketing_fivetran": marketing_fivetran_workspace, + "sales_fivetran": sales_fivetran_workspace, + }, +) diff --git a/examples/docs_snippets/docs_snippets/integrations/fivetran/representing_fivetran_assets.py b/examples/docs_snippets/docs_snippets/integrations/fivetran/representing_fivetran_assets.py new file mode 100644 index 0000000000000..7637330c175cd --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/fivetran/representing_fivetran_assets.py @@ -0,0 +1,12 @@ +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}) diff --git a/examples/docs_snippets/docs_snippets/integrations/fivetran/sync_and_materialize_fivetran_assets.py b/examples/docs_snippets/docs_snippets/integrations/fivetran/sync_and_materialize_fivetran_assets.py new file mode 100644 index 0000000000000..65a3490f59486 --- /dev/null +++ b/examples/docs_snippets/docs_snippets/integrations/fivetran/sync_and_materialize_fivetran_assets.py @@ -0,0 +1,16 @@ +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"), +) + +all_fivetran_assets = build_fivetran_assets_definitions(workspace=fivetran_workspace) + +defs = dg.Definitions( + assets=all_fivetran_assets, + resources={"fivetran": fivetran_workspace}, +) diff --git a/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/components/a_script_collection/components.yaml b/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/components/a_script_collection/components.yaml new file mode 100644 index 0000000000000..78badcc66c7cb --- /dev/null +++ b/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/components/a_script_collection/components.yaml @@ -0,0 +1,6 @@ +type: pipes_subprocess_script_collection + +params: + script_one: + assets: + - key: a \ No newline at end of file diff --git a/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/components/a_script_collection/defs.yml b/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/components/a_script_collection/defs.yml deleted file mode 100644 index 6ab7905f9eb5d..0000000000000 --- a/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/components/a_script_collection/defs.yml +++ /dev/null @@ -1,6 +0,0 @@ -component_type: pipes_subprocess_script_collection - -component_params: - script_one: - assets: - - key: a \ No newline at end of file diff --git a/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/definitions.py b/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/definitions.py index 46f9002ec61e6..b7a4a42f65ee3 100644 --- a/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/definitions.py +++ b/examples/experimental/components/examples/hello_world_deployment/code_locations/hello_world/hello_world/definitions.py @@ -2,7 +2,7 @@ import dagster as dg from dagster_components import ComponentRegistry, build_defs_from_toplevel_components_folder -from dagster_components.impls.pipes_subprocess_script_collection import ( +from dagster_components.lib.pipes_subprocess_script_collection import ( PipesSubprocessScriptCollection, ) diff --git a/examples/project_du_dbt_starter/.gitignore b/examples/project_du_dbt_starter/.gitignore index 2eea525d885d5..4aaa9f8182731 100644 --- a/examples/project_du_dbt_starter/.gitignore +++ b/examples/project_du_dbt_starter/.gitignore @@ -1 +1,186 @@ -.env \ No newline at end of file +.venv/ +.vscode/ +bin/ +__pycache__ +pyvenv.cfg +*.egg-info +.ipynb_checkpoints/ +.bashrc +.DS_STORE + +# Byte-compiled / optimized / DLL files +__pycache__/ +*.py[cod] +*$py.class + +# C extensions +*.so + +# Distribution / packaging +.Python +build/ +develop-eggs/ +dist/ +downloads/ +eggs/ +.eggs/ +lib/ +lib64/ +parts/ +sdist/ +var/ +wheels/ +share/python-wheels/ +*.egg-info/ +.installed.cfg +*.egg +MANIFEST + +# PyInstaller +# Usually these files are written by a python script from a template +# before PyInstaller builds the exe, so as to inject date/other infos into it. +*.manifest +*.spec + +# Installer logs +pip-log.txt +pip-delete-this-directory.txt + +# Unit test / coverage reports +htmlcov/ +.tox/ +.nox/ +.coverage +.coverage.* +.cache +nosetests.xml +coverage.xml +*.cover +*.py,cover +.hypothesis/ +.pytest_cache/ +cover/ + +# Translations +*.mo +*.pot + +# Django stuff: +*.log +local_settings.py +db.sqlite3 +db.sqlite3-journal + +# Flask stuff: +instance/ +.webassets-cache + +# Scrapy stuff: +.scrapy + +# Sphinx documentation +docs/_build/ + +# PyBuilder +.pybuilder/ +target/ + +# Jupyter Notebook +.ipynb_checkpoints + +# IPython +profile_default/ +ipython_config.py + +# pyenv +# For a library or package, you might want to ignore these files since the code is +# intended to run in multiple environments; otherwise, check them in: +# .python-version + +# pipenv +# According to pypa/pipenv#598, it is recommended to include Pipfile.lock in version control. +# However, in case of collaboration, if having platform-specific dependencies or dependencies +# having no cross-platform support, pipenv may install dependencies that don't work, or not +# install all needed dependencies. +#Pipfile.lock + +# poetry +# Similar to Pipfile.lock, it is generally recommended to include poetry.lock in version control. +# This is especially recommended for binary packages to ensure reproducibility, and is more +# commonly ignored for libraries. +# https://python-poetry.org/docs/basic-usage/#commit-your-poetrylock-file-to-version-control +#poetry.lock + +# pdm +# Similar to Pipfile.lock, it is generally recommended to include pdm.lock in version control. +#pdm.lock +# pdm stores project-wide configurations in .pdm.toml, but it is recommended to not include it +# in version control. +# https://pdm.fming.dev/#use-with-ide +.pdm.toml + +# PEP 582; used by e.g. github.com/David-OConnor/pyflow and github.com/pdm-project/pdm +__pypackages__/ + +# Celery stuff +celerybeat-schedule +celerybeat.pid + +# SageMath parsed files +*.sage.py + +# Environments +.env +.venv +env/ +venv/ +ENV/ +env.bak/ +venv.bak/ + +# Spyder project settings +.spyderproject +.spyproject + +# Rope project settings +.ropeproject + +# mkdocs documentation +/site + +# mypy +.mypy_cache/ +.dmypy.json +dmypy.json + +# Pyre type checker +.pyre/ + +# pytype static type analyzer +.pytype/ + +# Cython debug symbols +cython_debug/ + +# PyCharm +# JetBrains specific template is maintained in a separate JetBrains.gitignore that can +# be found at https://github.com/github/gitignore/blob/main/Global/JetBrains.gitignore +# and can be added to the global gitignore or merged into this file. For a more nuclear +# option (not recommended) you can uncomment the following to ignore the entire idea folder. +#.idea/ + +tmp*/ + +*.duckdb +*.geojson +*.csv +*.parquet + +data/outputs/* +data/raw/* +data/requests/*.json +data/staging/* + +*.user.yml + +*/package-lock.yml diff --git a/integration_tests/test_suites/daemon-test-suite/auto_run_reexecution_tests/test_auto_run_reexecution.py b/integration_tests/test_suites/daemon-test-suite/auto_run_reexecution_tests/test_auto_run_reexecution.py index 2aa4da7b2152c..49c839775fbb4 100644 --- a/integration_tests/test_suites/daemon-test-suite/auto_run_reexecution_tests/test_auto_run_reexecution.py +++ b/integration_tests/test_suites/daemon-test-suite/auto_run_reexecution_tests/test_auto_run_reexecution.py @@ -1,6 +1,6 @@ import logging import time -from typing import cast +from typing import Sequence, cast from unittest.mock import PropertyMock, patch from dagster import DagsterEvent, DagsterEventType, DagsterInstance, EventLogEntry @@ -9,7 +9,7 @@ from dagster._core.execution.api import create_execution_plan from dagster._core.execution.plan.resume_retry import ReexecutionStrategy from dagster._core.snap import snapshot_from_execution_plan -from dagster._core.storage.dagster_run import DagsterRunStatus, RunsFilter +from dagster._core.storage.dagster_run import DagsterRun, DagsterRunStatus, RunsFilter from dagster._core.storage.tags import ( AUTO_RETRY_RUN_ID_TAG, MAX_RETRIES_TAG, @@ -48,6 +48,21 @@ def create_run(instance, **kwargs): ) +def process_runs_in_queue(instance, queue: Sequence[DagsterRun]): + """The MockedRunCoordinator doesn't update the status of a run when submit_run is called. All other + RunCoordinators move the run from NOT_STARTED to another status, so post-process the queue to update the + status of each run to QUEUED, since the auto-reexecution logic checks the status of the runs. + """ + for run in queue: + updated_run = instance.get_run_by_id(run.run_id) + if updated_run.status == DagsterRunStatus.NOT_STARTED: + launch_started_event = DagsterEvent( + event_type_value=DagsterEventType.PIPELINE_STARTING.value, + job_name=run.job_name, + ) + instance.report_dagster_event(launch_started_event, run_id=run.run_id) + + def test_filter_runs_to_should_retry(instance): max_retries_setting = 2 instance.wipe() @@ -62,7 +77,7 @@ def test_filter_runs_to_should_retry(instance): run = create_run(instance, status=DagsterRunStatus.STARTED) - assert list(filter_runs_to_should_retry([run], instance, max_retries_setting)) == [] + assert list(filter_runs_to_should_retry([run], instance)) == [] dagster_event = DagsterEvent( event_type_value=DagsterEventType.PIPELINE_FAILURE.value, @@ -89,7 +104,6 @@ def test_filter_runs_to_should_retry(instance): filter_runs_to_should_retry( instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), instance, - max_retries_setting, ) ) ) @@ -111,7 +125,7 @@ def test_filter_runs_no_retry_on_asset_or_op_failure(instance_no_retry_on_asset_ run = create_run(instance, status=DagsterRunStatus.STARTED, tags={MAX_RETRIES_TAG: "2"}) - assert list(filter_runs_to_should_retry([run], instance, max_retries_setting)) == [] + assert list(filter_runs_to_should_retry([run], instance)) == [] dagster_event = DagsterEvent( event_type_value=DagsterEventType.PIPELINE_FAILURE.value, @@ -133,7 +147,6 @@ def test_filter_runs_no_retry_on_asset_or_op_failure(instance_no_retry_on_asset_ filter_runs_to_should_retry( instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), instance, - max_retries_setting, ) ) ) @@ -172,7 +185,6 @@ def test_filter_runs_no_retry_on_asset_or_op_failure(instance_no_retry_on_asset_ filter_runs_to_should_retry( instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), instance, - max_retries_setting, ) ) ) @@ -203,7 +215,6 @@ def test_filter_runs_no_retry_on_asset_or_op_failure(instance_no_retry_on_asset_ filter_runs_to_should_retry( instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), instance, - max_retries_setting, ) ) assert len(runs_to_retry) == 1 @@ -223,7 +234,7 @@ def test_filter_runs_to_should_retry_tags(instance): run = create_run(instance, status=DagsterRunStatus.STARTED, tags={MAX_RETRIES_TAG: "0"}) - assert list(filter_runs_to_should_retry([run], instance, max_retries_setting)) == [] + assert list(filter_runs_to_should_retry([run], instance)) == [] instance.report_run_failed(run) run = instance.get_run_by_id(run.run_id) @@ -236,7 +247,6 @@ def test_filter_runs_to_should_retry_tags(instance): filter_runs_to_should_retry( instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), instance, - max_retries_setting, ) ) ) @@ -256,7 +266,7 @@ def test_filter_runs_to_should_retry_tags(instance): run = create_run(instance, status=DagsterRunStatus.STARTED, tags={MAX_RETRIES_TAG: "10"}) - assert list(filter_runs_to_should_retry([run], instance, max_retries_setting)) == [] + assert list(filter_runs_to_should_retry([run], instance)) == [] instance.report_run_failed(run) run = instance.get_run_by_id(run.run_id) @@ -269,7 +279,6 @@ def test_filter_runs_to_should_retry_tags(instance): filter_runs_to_should_retry( instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), instance, - max_retries_setting, ) ) ) @@ -291,7 +300,7 @@ def test_filter_runs_to_should_retry_tags(instance): instance, status=DagsterRunStatus.STARTED, tags={MAX_RETRIES_TAG: "not-an-int"} ) - assert list(filter_runs_to_should_retry([run], instance, max_retries_setting)) == [] + assert list(filter_runs_to_should_retry([run], instance)) == [] instance.report_run_failed(run) run = instance.get_run_by_id(run.run_id) @@ -303,13 +312,76 @@ def test_filter_runs_to_should_retry_tags(instance): filter_runs_to_should_retry( instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), instance, - max_retries_setting, ) ) == [] ) +def test_fallback_to_computing_should_retry_if_tag_not_present(instance): + max_retries_setting = 2 + instance.wipe() + with patch( + instance.__class__.__module__ + + "." + + instance.__class__.__name__ + + ".run_retries_max_retries", + new_callable=PropertyMock, + ) as mock_max_run_retries: + mock_max_run_retries.return_value = max_retries_setting + with patch( + instance.__class__.__module__ + + "." + + instance.__class__.__name__ + + ".run_retries_enabled", + new_callable=PropertyMock, + ) as mock_run_retries_enabled: + # mock the return of run_retries_enabled to False so that the WILL_RETRY tag is not set on + # the run + mock_run_retries_enabled.return_value = False + + run = create_run(instance, status=DagsterRunStatus.STARTED) + + assert list(filter_runs_to_should_retry([run], instance)) == [] + + dagster_event = DagsterEvent( + event_type_value=DagsterEventType.PIPELINE_FAILURE.value, + job_name="foo", + message="", + ) + event_record = EventLogEntry( + user_message="", + level=logging.ERROR, + job_name="foo", + run_id=run.run_id, + error_info=None, + timestamp=time.time(), + dagster_event=dagster_event, + ) + instance.handle_new_event(event_record) + run = instance.get_run_by_id(run.run_id) + assert run + assert run.tags.get(WILL_RETRY_TAG) is None + + # filter_runs_to_should_retry should see that the WILL_RETRY tag is not set and recompute + # if the run should be retried. It will also add the tag + assert ( + len( + list( + filter_runs_to_should_retry( + instance.get_runs(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), + instance, + ) + ) + ) + == 1 + ) + + run = instance.get_run_by_id(run.run_id) + assert run + assert run.tags.get(WILL_RETRY_TAG) == "true" + + def test_consume_new_runs_for_automatic_reexecution(instance, workspace_context): instance.wipe() instance.run_coordinator.queue().clear() @@ -350,7 +422,8 @@ def test_consume_new_runs_for_automatic_reexecution(instance, workspace_context) ) ) assert len(instance.run_coordinator.queue()) == 1 - first_retry = instance.run_coordinator.queue()[0] + process_runs_in_queue(instance, instance.run_coordinator.queue()) + first_retry = instance.get_run_by_id(instance.run_coordinator.queue()[0].run_id) run = instance.get_run_by_id(run.run_id) assert run.tags.get(AUTO_RETRY_RUN_ID_TAG) == first_retry.run_id @@ -390,7 +463,8 @@ def test_consume_new_runs_for_automatic_reexecution(instance, workspace_context) ) ) assert len(instance.run_coordinator.queue()) == 2 - second_retry = instance.run_coordinator.queue()[1] + process_runs_in_queue(instance, instance.run_coordinator.queue()) + second_retry = instance.get_run_by_id(instance.run_coordinator.queue()[1].run_id) first_retry = instance.get_run_by_id(first_retry.run_id) assert first_retry.tags.get(AUTO_RETRY_RUN_ID_TAG) == second_retry.run_id @@ -428,10 +502,11 @@ def test_consume_new_runs_for_automatic_reexecution(instance, workspace_context) def test_consume_new_runs_for_automatic_reexecution_mimic_daemon_fails_before_run_is_launched( instance, workspace_context ): - """This test documents the behavior for an edge case where the daemon fails between creating the - run (instance.create_reexecuted_run) and submitting the run (instance.submit_run). The current - implementation of the daemon does not gracefully handle this case and the run will remain stuck - in a NOT_STARTED status. + """Tests that the daemon can recover if it fails between creating the retry run + (instance.create_reexecuted_run) and submitting the run (instance.submit_run). If the daemon fails + between these two calls, the cursors will not be updated, and the original run will be processed again. + This test asserts that the daemon is able to find the run that was created in the previous iteration + and then submits it. """ instance.wipe() instance.run_coordinator.queue().clear() @@ -478,6 +553,7 @@ def test_consume_new_runs_for_automatic_reexecution_mimic_daemon_fails_before_ru instance.run_coordinator.queue().clear() assert len(instance.run_coordinator.queue()) == 0 assert len(instance.get_runs()) == 2 # original run and retry run are both in the db + assert instance.get_run_by_id(first_retry_run_id).status == DagsterRunStatus.NOT_STARTED list( consume_new_runs_for_automatic_reexecution( @@ -488,10 +564,90 @@ def test_consume_new_runs_for_automatic_reexecution_mimic_daemon_fails_before_ru # because the retried run already exists, it is in the run group for the original run. Because # it is in the run group of the original run, we will see that the run group contains a run with - # parent_run_id as the original run. Therefore the original run will not be retried - # This is a problem, since the retried run will be stuck in a NOT_STARTED state + # parent_run_id as the original run. Then the daemon will check that the run has been started and + # submit the run if not. Since the first retried_run is not started, it will get resubmitted. + assert len(instance.run_coordinator.queue()) == 1 + second_retry = instance.run_coordinator.queue()[0] + run = instance.get_run_by_id(run.run_id) + assert run.tags.get(AUTO_RETRY_RUN_ID_TAG) == second_retry.run_id + # the second retry isn't a unique run, it's just a re-submission of the first run + assert second_retry.run_id == first_retry_run_id + + +def test_consume_new_runs_for_automatic_reexecution_mimic_daemon_fails_before_retry_tag_is_added( + instance, workspace_context +): + instance.wipe() + instance.run_coordinator.queue().clear() + list( + consume_new_runs_for_automatic_reexecution( + workspace_context, + instance.get_run_records(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), + ) + ) + assert len(instance.run_coordinator.queue()) == 0 - assert instance.get_run_by_id(first_retry_run_id).status == DagsterRunStatus.NOT_STARTED + + run = create_run(instance, status=DagsterRunStatus.STARTED, tags={MAX_RETRIES_TAG: "2"}) + dagster_event = DagsterEvent( + event_type_value=DagsterEventType.PIPELINE_FAILURE.value, + job_name="foo", + message="", + ) + event_record = EventLogEntry( + user_message="", + level=logging.ERROR, + job_name="foo", + run_id=run.run_id, + error_info=None, + timestamp=time.time(), + dagster_event=dagster_event, + ) + instance.handle_new_event(event_record) + run = instance.get_run_by_id(run.run_id) + assert run + assert run.tags.get(WILL_RETRY_TAG) == "true" + + list( + consume_new_runs_for_automatic_reexecution( + workspace_context, + instance.get_run_records(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), + ) + ) + assert len(instance.run_coordinator.queue()) == 1 + first_retry = instance.run_coordinator.queue()[0] + + # if the daemon fails between creating the run and adding the AUTO_RETRY_RUN_ID_TAG, the run will exist in the + # db but not the run_coordinator queue + instance.run_coordinator.queue().clear() + assert len(instance.run_coordinator.queue()) == 0 + assert len(instance.get_runs()) == 2 # original run and retry run are both in the db + # update the run to remove the AUTO_RETRY_RUN_ID_TAG to simulate the daemon failing before adding the tag + run = instance.get_run_by_id(run.run_id) + instance.delete_run(run.run_id) + run_tags = run.tags + del run_tags[AUTO_RETRY_RUN_ID_TAG] + run = run.with_tags(run_tags) + instance.add_run(run) + + assert len(instance.get_runs()) == 2 # original run and retry run are both in the db + + list( + consume_new_runs_for_automatic_reexecution( + workspace_context, + instance.get_run_records(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), + ) + ) + + # because the retried run already exists, it is in the run group for the original run. Because + # it is in the run group of the original run, we will see that the run group contains a run with + # parent_run_id as the original run. Then the daemon will check that the run has been started and + # submit the run if not. Since the first retried_run is not started, it will get resubmitted. + assert len(instance.run_coordinator.queue()) == 1 + second_retry = instance.run_coordinator.queue()[0] + run = instance.get_run_by_id(run.run_id) + assert run.tags.get(AUTO_RETRY_RUN_ID_TAG) == second_retry.run_id + assert second_retry.run_id == first_retry.run_id def test_consume_new_runs_for_automatic_reexecution_retry_run_deleted(instance, workspace_context): @@ -540,6 +696,8 @@ def test_consume_new_runs_for_automatic_reexecution_retry_run_deleted(instance, ) assert len(instance.run_coordinator.queue()) == 1 first_retry = instance.run_coordinator.queue()[0] + run = instance.get_run_by_id(run.run_id) + assert run.tags.get(AUTO_RETRY_RUN_ID_TAG) == first_retry.run_id # delete the run and remove it from the queue instance.delete_run(first_retry.run_id) @@ -555,9 +713,21 @@ def test_consume_new_runs_for_automatic_reexecution_retry_run_deleted(instance, ) assert len(instance.run_coordinator.queue()) == 1 + second_retry = instance.run_coordinator.queue()[0] + run = instance.get_run_by_id(run.run_id) + assert run.tags.get(AUTO_RETRY_RUN_ID_TAG) == second_retry.run_id def test_code_location_unavailable(instance, workspace_context): + """This test documents the behavior of the daemon if retry_run raises an exception. Since we + catch any exception raised by retry_run and log the error instead of re-raising, the daemon will + update the cursors and continue as normal. This means that if retry_run fails, the run will not + get processed again by the daemon. So we need to mark that the run will not be retried so that + the tags reflect the state of the system. + + If we modify the EventLogConsumerDaemon so that we can re-process runs where an error was raised, then + we could reconsider this behavior and try to reprocess runs where something in retry_run failed. + """ instance.wipe() instance.run_coordinator.queue().clear() list( @@ -604,6 +774,9 @@ def raise_code_unreachable_error(*args, **kwargs): ) assert len(instance.run_coordinator.queue()) == 0 + run = instance.get_run_by_id(run.run_id) + assert run + assert run.tags.get(WILL_RETRY_TAG) == "false" def test_consume_new_runs_for_automatic_reexecution_with_manual_retry(instance, workspace_context): @@ -640,7 +813,7 @@ def test_consume_new_runs_for_automatic_reexecution_with_manual_retry(instance, instance.handle_new_event(event_record) # before the auto-reexecution daemon can run, a user manually launches a retry - create_run( + manual_retry = create_run( instance, status=DagsterRunStatus.STARTED, parent_run_id=run.run_id, @@ -654,10 +827,13 @@ def test_consume_new_runs_for_automatic_reexecution_with_manual_retry(instance, instance.get_run_records(filters=RunsFilter(statuses=[DagsterRunStatus.FAILURE])), ) ) - # because the manually retried run already exists, it is in the run group for the original run. Because - # it is in the run group of the original run, we will see that the run group contains a run with - # parent_run_id as the original run. Therefore the original run will not be retried - assert len(instance.run_coordinator.queue()) == 0 + # the daemon can distinguish between a manual retry of a run and an auto-retry of a run and still + # launch an automatic retry. Therefore the original run will be retried + assert len(instance.run_coordinator.queue()) == 1 + retry_run = instance.run_coordinator.queue()[0] + run = instance.get_run_by_id(run.run_id) + assert run.tags.get(AUTO_RETRY_RUN_ID_TAG) == retry_run.run_id + assert run.tags.get(AUTO_RETRY_RUN_ID_TAG) != manual_retry.run_id def test_daemon_enabled(instance): diff --git a/js_modules/dagster-ui/packages/app-oss/tsconfig.json b/js_modules/dagster-ui/packages/app-oss/tsconfig.json index fc2d305c048ae..5345f969e6b42 100644 --- a/js_modules/dagster-ui/packages/app-oss/tsconfig.json +++ b/js_modules/dagster-ui/packages/app-oss/tsconfig.json @@ -3,6 +3,7 @@ "paths": { "@dagster-io/ui-core/*": ["../ui-core/src/*"], "@dagster-io/ui-components": ["../ui-components/src"], + "@dagster-io/ui-components/editor": ["../ui-components/src/editor"], "shared/*": ["../ui-core/src/*"] }, "module": "esnext", diff --git a/js_modules/dagster-ui/packages/ui-components/CHANGES.md b/js_modules/dagster-ui/packages/ui-components/CHANGES.md index 17f3c1cdafd7d..52e39450f4a98 100644 --- a/js_modules/dagster-ui/packages/ui-components/CHANGES.md +++ b/js_modules/dagster-ui/packages/ui-components/CHANGES.md @@ -1,3 +1,7 @@ +# 1.2.1 (December 5, 2024) + +- Split CodeMirror-based components out of main index export + # 1.2.0 (December 5, 2024) - Migrate styled-components to v6 diff --git a/js_modules/dagster-ui/packages/ui-components/package.json b/js_modules/dagster-ui/packages/ui-components/package.json index b178ac7801105..ca29def1f1bc4 100644 --- a/js_modules/dagster-ui/packages/ui-components/package.json +++ b/js_modules/dagster-ui/packages/ui-components/package.json @@ -1,6 +1,6 @@ { "name": "@dagster-io/ui-components", - "version": "1.2.0", + "version": "1.2.1", "description": "Dagster UI Component Library", "license": "Apache-2.0", "main": "lib/index.js", diff --git a/js_modules/dagster-ui/packages/ui-components/rollup.config.js b/js_modules/dagster-ui/packages/ui-components/rollup.config.js index 10d98f392aca3..c20bcb32acb23 100644 --- a/js_modules/dagster-ui/packages/ui-components/rollup.config.js +++ b/js_modules/dagster-ui/packages/ui-components/rollup.config.js @@ -11,52 +11,68 @@ import styles from 'rollup-plugin-styles'; const extensions = ['.js', '.jsx', '.ts', '.tsx', '.css', '.svg']; -export default { - input: { - index: './src/index.ts', +const sharedPlugins = [ + styles({ + extract: true, + }), + json(), + url(), + image(), + babel({ + babelHelpers: 'bundled', + exclude: 'node_modules/**', + extensions: ['.js', '.jsx', '.ts', '.tsx'], + }), + commonjs(), + polyfills(), + resolve({extensions, preferBuiltins: false}), +]; - // Our core fonts, usable as global style components, e.g. ``. - 'fonts/GlobalGeistMono': './src/fonts/GlobalGeistMono.tsx', - 'fonts/GlobalGeist': './src/fonts/GlobalGeist.tsx', +const sharedExternals = [ + '@blueprintjs/core', + '@blueprintjs/popover2', + '@blueprintjs/select', + 'react', + 'react-dom', + 'react-is', + 'styled-components', + 'stylis', +]; - // Components are listed here individually so that they may be imported - // without pulling in the entire library. - 'components/Box': './src/components/Box.tsx', - 'components/Button': './src/components/Button.tsx', - 'components/Color': './src/components/Color.tsx', - 'components/Icon': './src/components/Icon.tsx', +export default [ + { + input: { + index: './src/index.ts', + + // Our core fonts, usable as global style components, e.g. ``. + 'fonts/GlobalGeistMono': './src/fonts/GlobalGeistMono.tsx', + 'fonts/GlobalGeist': './src/fonts/GlobalGeist.tsx', + + // Components are listed here individually so that they may be imported + // without pulling in the entire library. + 'components/Box': './src/components/Box.tsx', + 'components/Button': './src/components/Button.tsx', + 'components/Color': './src/components/Color.tsx', + 'components/Icon': './src/components/Icon.tsx', + }, + output: { + dir: 'lib', + exports: 'named', + format: 'cjs', + sourcemap: true, + }, + plugins: sharedPlugins, + external: [...sharedExternals, '@tanstack/react-virtual'], }, - output: { - dir: 'lib', - exports: 'named', - format: 'cjs', - sourcemap: true, + { + input: './src/editor.ts', + output: { + dir: 'lib', + exports: 'named', + format: 'cjs', + sourcemap: true, + }, + plugins: sharedPlugins, + external: sharedExternals, }, - plugins: [ - styles({ - extract: true, - }), - json(), - url(), - image(), - babel({ - babelHelpers: 'bundled', - exclude: 'node_modules/**', - extensions: ['.js', '.jsx', '.ts', '.tsx'], - }), - commonjs(), - polyfills(), - resolve({extensions, preferBuiltins: false}), - ], - external: [ - '@blueprintjs/core', - '@blueprintjs/popover2', - '@blueprintjs/select', - '@tanstack/react-virtual', - 'react', - 'react-dom', - 'react-is', - 'styled-components', - 'stylis', - ], -}; +]; diff --git a/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx b/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx index b5e4763261e91..e2326328b9cd2 100644 --- a/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx +++ b/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx @@ -101,6 +101,7 @@ import dagster_reversed from '../icon-svgs/dagster_reversed.svg'; import dagster_solid from '../icon-svgs/dagster_solid.svg'; import dagsterlabs from '../icon-svgs/dagsterlabs.svg'; import dash from '../icon-svgs/dash.svg'; +import data_object from '../icon-svgs/data_object.svg'; import data_reliability from '../icon-svgs/data_reliability.svg'; import data_type from '../icon-svgs/data_type.svg'; import database from '../icon-svgs/database.svg'; @@ -268,6 +269,7 @@ import password from '../icon-svgs/password.svg'; import pause from '../icon-svgs/pause.svg'; import people from '../icon-svgs/people.svg'; import plots from '../icon-svgs/plots.svg'; +import preview_tick from '../icon-svgs/preview_tick.svg'; import priority_1 from '../icon-svgs/priority_1.svg'; import priority_2 from '../icon-svgs/priority_2.svg'; import priority_3 from '../icon-svgs/priority_3.svg'; @@ -502,6 +504,7 @@ export const Icons = { dash, data_reliability, data_type, + data_object, database, datatype_array, datatype_bool, @@ -664,6 +667,7 @@ export const Icons = { pause, people, plots, + preview_tick, priority_1, priority_2, priority_3, diff --git a/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx b/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx index 6931dabab7d1e..037f355b399ce 100644 --- a/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx +++ b/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx @@ -1,4 +1,5 @@ import * as React from 'react'; +import styled from 'styled-components'; import {Box} from './Box'; import {Colors} from './Color'; @@ -27,7 +28,7 @@ export const NonIdealState = ({ const singleContentElement = [title, description, action].filter(Boolean).length === 1; return ( - {description}} {action} - + ); }; + +export const NonIdealStateWrapper = styled(Box)``; diff --git a/js_modules/dagster-ui/packages/ui-components/src/editor.ts b/js_modules/dagster-ui/packages/ui-components/src/editor.ts new file mode 100644 index 0000000000000..62a654c369f70 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-components/src/editor.ts @@ -0,0 +1,4 @@ +export * from './components/ConfigEditorDialog'; +export * from './components/ConfigEditorWithSchema'; +export * from './components/NewConfigEditor'; +export * from './components/StyledRawCodeMirror'; diff --git a/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/data_object.svg b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/data_object.svg new file mode 100644 index 0000000000000..8199c504d3bf9 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/data_object.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/preview_tick.svg b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/preview_tick.svg new file mode 100644 index 0000000000000..1f6680536c263 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/preview_tick.svg @@ -0,0 +1,4 @@ + + + + diff --git a/js_modules/dagster-ui/packages/ui-components/src/index.ts b/js_modules/dagster-ui/packages/ui-components/src/index.ts index 1224d069e9637..ba72a801271cf 100644 --- a/js_modules/dagster-ui/packages/ui-components/src/index.ts +++ b/js_modules/dagster-ui/packages/ui-components/src/index.ts @@ -7,8 +7,6 @@ export * from './components/ButtonGroup'; export * from './components/ButtonLink'; export * from './components/Checkbox'; export * from './components/CollapsibleSection'; -export * from './components/ConfigEditorDialog'; -export * from './components/ConfigEditorWithSchema'; export * from './components/ConfigTypeSchema'; export * from './components/Countdown'; export * from './components/CursorControls'; @@ -25,7 +23,6 @@ export * from './components/MainContent'; export * from './components/Menu'; export * from './components/MetadataTable'; export * from './components/MiddleTruncate'; -export * from './components/NewConfigEditor'; export * from './components/NonIdealState'; export * from './components/Page'; export * from './components/PageHeader'; @@ -41,7 +38,6 @@ export * from './components/Spinner'; export * from './components/SpinnerWithText'; export * from './components/SplitPanelContainer'; export * from './components/StyledButton'; -export * from './components/StyledRawCodeMirror'; export * from './components/SubwayDot'; export * from './components/Suggest'; export * from './components/Table'; diff --git a/js_modules/dagster-ui/packages/ui-core/client.json b/js_modules/dagster-ui/packages/ui-core/client.json index 4c62310e025b4..9b8c430dd2ead 100644 --- a/js_modules/dagster-ui/packages/ui-core/client.json +++ b/js_modules/dagster-ui/packages/ui-core/client.json @@ -126,6 +126,7 @@ "RunRootQuery": "1aa4561b33c2cfb079d7a3ff284096fc3208a46dee748a24c7af827a2cb22919", "RunStatsQuery": "75e80f740a79607de9e1152f9b7074d319197fbc219784c767c1abd5553e9a49", "LaunchPipelineExecution": "292088c4a697aca6be1d3bbc0cfc45d8a13cdb2e75cfedc64b68c6245ea34f89", + "LaunchMultipleRuns": "a56d9efdb35e71e0fd1744dd768129248943bc5b23e717458b82c46829661763", "Delete": "3c61c79b99122910e754a8863e80dc5ed479a0c23cc1a9d9878d91e603fc0dfe", "Terminate": "67acf403eb320a93c9a9aa07f675a1557e0887d499cd5598f1d5ff360afc15c0", "LaunchPipelineReexecution": "d21e4ecaf3d1d163c4772f1d847dbdcbdaa9a40e6de0808a064ae767adf0c311", diff --git a/js_modules/dagster-ui/packages/ui-core/jest.config.js b/js_modules/dagster-ui/packages/ui-core/jest.config.js index 3039f576f7a91..543a7785cd2aa 100644 --- a/js_modules/dagster-ui/packages/ui-core/jest.config.js +++ b/js_modules/dagster-ui/packages/ui-core/jest.config.js @@ -78,6 +78,7 @@ module.exports = { '\\.(css|less)$': 'identity-obj-proxy', '^worker-loader(.*)/workers/(.*)$': '/jest/mocks/$2', '^@dagster-io/ui-components$': '/../ui-components/src/index', + '^@dagster-io/ui-components/editor$': '/../ui-components/src/editor', '^shared/(.*)$': '/src/$1', }, diff --git a/js_modules/dagster-ui/packages/ui-core/jest/mocks/ComputeGraphData.worker.ts b/js_modules/dagster-ui/packages/ui-core/jest/mocks/ComputeGraphData.worker.ts new file mode 100644 index 0000000000000..53c25b23e987d --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/jest/mocks/ComputeGraphData.worker.ts @@ -0,0 +1,51 @@ +import {setFeatureFlagsInternal} from '../../src/app/Flags'; +import {assertUnreachable} from '../../src/app/Util'; +import {computeGraphData} from '../../src/asset-graph/ComputeGraphData'; +import {ComputeGraphDataWorkerMessageType} from '../../src/asset-graph/ComputeGraphData.types'; +import {buildGraphData} from '../../src/asset-graph/Utils'; + +// eslint-disable-next-line import/no-default-export +export default class MockWorker { + onmessage: Array<(data: any) => void> = []; + + addEventListener(_type: string, handler: any) { + this.onmessage.push(handler); + } + + removeEventListener(_type: string, handler: any) { + const index = this.onmessage.indexOf(handler); + if (index !== -1) { + this.onmessage.splice(index, 1); + } + } + + // mock expects data: { } instead of e: { data: { } } + async postMessage(data: ComputeGraphDataWorkerMessageType) { + if (data.type === 'computeGraphData') { + if (data.flagAssetSelectionSyntax) { + setFeatureFlagsInternal({flagAssetSelectionSyntax: true}); + } + const state = await computeGraphData(data); + this.onmessage.forEach((onmessage) => onmessage({data: {...state, id: data.id}})); + } else if (data.type === 'buildGraphData') { + this.onmessage.forEach((onmessage) => + onmessage({data: {...buildGraphData(data.nodes), id: data.id}}), + ); + } else { + assertUnreachable(data); + } + } +} + +const originalWorker = global.Worker; +// @ts-expect-error - test shenanigans +global.Worker = function ComputeGraphDataMockWorkerWrapper( + url: string | URL, + opts?: WorkerOptions, +) { + if (url.toString().endsWith('ComputeGraphData.worker')) { + return new MockWorker(); + } else { + return new originalWorker(url, opts); + } +}; diff --git a/js_modules/dagster-ui/packages/ui-core/package.json b/js_modules/dagster-ui/packages/ui-core/package.json index 23d40dd9517ab..a17017e2ce43d 100644 --- a/js_modules/dagster-ui/packages/ui-core/package.json +++ b/js_modules/dagster-ui/packages/ui-core/package.json @@ -53,6 +53,7 @@ "dayjs": "^1.11.7", "deepmerge": "^4.2.2", "fake-indexeddb": "^4.0.2", + "fast-text-encoding": "^1.0.6", "fuse.js": "^6.4.2", "graphql": "^16.8.1", "graphql-codegen-persisted-query-ids": "^0.2.0", @@ -71,6 +72,7 @@ "rehype-sanitize": "^5.0.1", "remark": "^14.0.2", "remark-gfm": "3.0.1", + "spark-md5": "^3.0.2", "strip-markdown": "^6.0.0", "subscriptions-transport-ws": "^0.9.15", "worker-loader": "^3.0.8", @@ -117,6 +119,7 @@ "@types/color": "^3.0.2", "@types/dagre": "^0.7.42", "@types/faker": "^5.1.7", + "@types/fast-text-encoding": "^1.0.3", "@types/graphql": "^14.5.0", "@types/jest": "^29.5.11", "@types/lodash": "^4.14.145", @@ -127,6 +130,7 @@ "@types/react-dom": "^18.3.1", "@types/react-router": "^5.1.17", "@types/react-router-dom": "^5.3.3", + "@types/spark-md5": "^3", "@types/testing-library__jest-dom": "^5.14.2", "@types/ws": "^6.0.3", "@typescript-eslint/eslint-plugin": "^8.9.0", diff --git a/js_modules/dagster-ui/packages/ui-core/src/app/DefaultFeatureFlags.oss.tsx b/js_modules/dagster-ui/packages/ui-core/src/app/DefaultFeatureFlags.oss.tsx index cf5ec46efc12c..20590e2f50c18 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/app/DefaultFeatureFlags.oss.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/app/DefaultFeatureFlags.oss.tsx @@ -4,6 +4,8 @@ import {FeatureFlag} from 'shared/app/FeatureFlags.oss'; * Default values for feature flags when they are unset. */ export const DEFAULT_FEATURE_FLAG_VALUES: Partial> = { + [FeatureFlag.flagAssetSelectionWorker]: true, + // Flags for tests [FeatureFlag.__TestFlagDefaultTrue]: true, [FeatureFlag.__TestFlagDefaultFalse]: false, diff --git a/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx b/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx index 5a4a7512fb851..c9244178d1cfe 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx @@ -6,6 +6,7 @@ export enum FeatureFlag { flagLegacyRunsPage = 'flagLegacyRunsPage', flagAssetSelectionSyntax = 'flagAssetSelectionSyntax', flagRunSelectionSyntax = 'flagRunSelectionSyntax', + flagAssetSelectionWorker = 'flagAssetSelectionWorker', // Flags for tests __TestFlagDefaultNone = '__TestFlagDefaultNone', diff --git a/js_modules/dagster-ui/packages/ui-core/src/app/Flags.tsx b/js_modules/dagster-ui/packages/ui-core/src/app/Flags.tsx index ac9d1eed0ed50..978e1e2949c34 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/app/Flags.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/app/Flags.tsx @@ -28,7 +28,7 @@ const initializeFeatureFlags = () => { flags.forEach((flag: FeatureFlag) => { migratedFlags[flag] = true; }); - setFeatureFlagsInternal(migratedFlags, false); // Prevent broadcasting during migration + setFeatureFlagsInternal(migratedFlags); flags = migratedFlags; } @@ -37,16 +37,15 @@ const initializeFeatureFlags = () => { /** * Internal function to set feature flags without broadcasting. - * Used during initialization and migration. + * Used during initialization and migration and by web-workers. */ -const setFeatureFlagsInternal = (flags: FeatureFlagMap, broadcast: boolean = true) => { +export const setFeatureFlagsInternal = (flags: FeatureFlagMap) => { if (typeof flags !== 'object' || Array.isArray(flags)) { throw new Error('flags must be an object mapping FeatureFlag to boolean values'); } currentFeatureFlags = flags; - localStorage.setItem(DAGSTER_FLAGS_KEY, JSON.stringify(flags)); - if (broadcast) { - featureFlagsChannel.postMessage('updated'); + if (typeof localStorage !== 'undefined') { + localStorage.setItem(DAGSTER_FLAGS_KEY, JSON.stringify(flags)); } }; @@ -128,4 +127,5 @@ export const useFeatureFlags = (): Readonly> => { */ export const setFeatureFlags = (flags: FeatureFlagMap) => { setFeatureFlagsInternal(flags); + featureFlagsChannel.postMessage('updated'); }; diff --git a/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx b/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx index ce8133c042eb1..291afd68962fc 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx @@ -9,6 +9,7 @@ import {gql} from '../apollo-client'; export enum TelemetryAction { LAUNCH_RUN = 'LAUNCH_RUN', + LAUNCH_MULTIPLE_RUNS = 'LAUNCH_MULTIPLE_RUNS', GRAPHQL_QUERY_COMPLETED = 'GRAPHQL_QUERY_COMPLETED', } @@ -38,7 +39,7 @@ const LOG_TELEMETRY_MUTATION = gql` export async function logTelemetry( pathPrefix: string, action: TelemetryAction, - metadata: {[key: string]: string | null | undefined} = {}, + metadata: {[key: string]: string | string[] | null | undefined} = {}, ) { const graphqlPath = `${pathPrefix || ''}/graphql`; @@ -63,7 +64,10 @@ export async function logTelemetry( export const useTelemetryAction = () => { const {basePath, telemetryEnabled} = useContext(AppContext); return useCallback( - (action: TelemetryAction, metadata: {[key: string]: string | null | undefined} = {}) => { + ( + action: TelemetryAction, + metadata: {[key: string]: string | string[] | null | undefined} = {}, + ) => { if (telemetryEnabled) { logTelemetry(basePath, action, metadata); } diff --git a/js_modules/dagster-ui/packages/ui-core/src/app/Util.tsx b/js_modules/dagster-ui/packages/ui-core/src/app/Util.tsx index 663f422208a57..f09f482ea1555 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/app/Util.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/app/Util.tsx @@ -134,7 +134,7 @@ export function asyncMemoize Promise hashFn?: (arg: T, ...rest: any[]) => any, hashSize?: number, ): U { - const cache = new LRU(hashSize || 50); + const cache = new LRU(hashSize || 50); return (async (arg: T, ...rest: any[]) => { const key = hashFn ? hashFn(arg, ...rest) : arg; if (cache.has(key)) { @@ -160,6 +160,8 @@ export function indexedDBAsyncMemoize> = {}; + async function genHashKey(arg: T, ...rest: any[]) { const hash = hashFn ? hashFn(arg, ...rest) : arg; @@ -182,17 +184,21 @@ export function indexedDBAsyncMemoize { + const result = await fn(arg, ...rest); + // Resolve the promise before storing the result in IndexedDB + res(result); + if (lru) { + await lru.set(hashKey, result, { + // Some day in the year 2050... + expiry: new Date(9 ** 13), + }); + delete hashToPromise[hashKey]; + } }); } + resolve(await hashToPromise[hashKey]!); }); }) as any; ret.isCached = async (arg: T, ...rest: any) => { diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-data/__tests__/AssetLiveDataProvider.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/asset-data/__tests__/AssetLiveDataProvider.test.tsx index 2d8b826bebb17..5a1964a8f44bf 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/asset-data/__tests__/AssetLiveDataProvider.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-data/__tests__/AssetLiveDataProvider.test.tsx @@ -30,6 +30,19 @@ jest.mock('../../live-data-provider/util', () => { }; }); +jest.mock('../../live-data-provider/LiveDataScheduler', () => { + return { + LiveDataScheduler: class LiveDataScheduler { + scheduleStartFetchLoop(doStart: () => void) { + doStart(); + } + scheduleStopFetchLoop(doStop: () => void) { + doStop(); + } + }, + }; +}); + function Test({ mocks, hooks, diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/AssetGraphExplorer.tsx b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/AssetGraphExplorer.tsx index ed250aacd5f0d..7faacf10d2e73 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/AssetGraphExplorer.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/AssetGraphExplorer.tsx @@ -76,7 +76,7 @@ import { } from '../pipelines/GraphNotices'; import {ExplorerPath} from '../pipelines/PipelinePathUtils'; import {StaticSetFilter} from '../ui/BaseFilters/useStaticSetFilter'; -import {Loading} from '../ui/Loading'; +import {Loading, LoadingSpinner} from '../ui/Loading'; type AssetNode = AssetNodeForGraphQueryFragment; @@ -98,15 +98,24 @@ type Props = { export const MINIMAL_SCALE = 0.6; export const GROUPS_ONLY_SCALE = 0.15; +const DEFAULT_SET_HIDE_NODES_MATCH = (_node: AssetNodeForGraphQueryFragment) => true; + export const AssetGraphExplorer = (props: Props) => { const fullAssetGraphData = useFullAssetGraphData(props.fetchOptions); - const [hideNodesMatching, setHideNodesMatching] = useState( - () => (_node: AssetNodeForGraphQueryFragment) => true, - ); + const [hideNodesMatching, setHideNodesMatching] = useState(() => DEFAULT_SET_HIDE_NODES_MATCH); - const {fetchResult, assetGraphData, graphQueryItems, allAssetKeys} = useAssetGraphData( + const { + loading: graphDataLoading, + fetchResult, + assetGraphData, + graphQueryItems, + allAssetKeys, + } = useAssetGraphData( props.explorerPath.opsQuery, - {...props.fetchOptions, hideNodesMatching}, + useMemo( + () => ({...props.fetchOptions, hideNodesMatching}), + [props.fetchOptions, hideNodesMatching], + ), ); const {explorerPath, onChangeExplorerPath} = props; @@ -119,7 +128,7 @@ export const AssetGraphExplorer = (props: Props) => { () => (fullAssetGraphData ? Object.values(fullAssetGraphData.nodes) : []), [fullAssetGraphData], ), - loading: fetchResult.loading, + loading: graphDataLoading, viewType: props.viewType, assetSelection: explorerPath.opsQuery, setAssetSelection: React.useCallback( @@ -143,6 +152,9 @@ export const AssetGraphExplorer = (props: Props) => { return ( {() => { + if (graphDataLoading || filteredAssetsLoading) { + return ; + } if (!assetGraphData || !allAssetKeys || !fullAssetGraphData) { return ; } @@ -169,7 +181,7 @@ export const AssetGraphExplorer = (props: Props) => { filterButton={button} kindFilter={kindFilter} groupsFilter={groupsFilter} - filteredAssetsLoading={filteredAssetsLoading} + loading={filteredAssetsLoading || graphDataLoading} {...props} /> ); @@ -183,7 +195,7 @@ type WithDataProps = Props & { assetGraphData: GraphData; fullAssetGraphData: GraphData; graphQueryItems: AssetGraphQueryItem[]; - filteredAssetsLoading: boolean; + loading: boolean; filterButton: React.ReactNode; filterBar: React.ReactNode; @@ -209,7 +221,7 @@ const AssetGraphExplorerWithData = ({ viewType, kindFilter, groupsFilter, - filteredAssetsLoading, + loading: dataLoading, }: WithDataProps) => { const findAssetLocation = useFindAssetLocation(); const [highlighted, setHighlighted] = React.useState(null); @@ -235,7 +247,11 @@ const AssetGraphExplorerWithData = ({ }); const focusGroupIdAfterLayoutRef = React.useRef(''); - const {layout, loading, async} = useAssetLayout( + const { + layout, + loading: layoutLoading, + async, + } = useAssetLayout( assetGraphData, expandedGroups, useMemo(() => ({direction}), [direction]), @@ -665,6 +681,8 @@ const AssetGraphExplorerWithData = ({ ) : null; + const loading = layoutLoading || dataLoading; + const explorer = ( Loading assets… @@ -769,7 +787,7 @@ const AssetGraphExplorerWithData = ({ ) } second={ - filteredAssetsLoading ? null : selectedGraphNodes.length === 1 && selectedGraphNodes[0] ? ( + loading ? null : selectedGraphNodes.length === 1 && selectedGraphNodes[0] ? ( @@ -814,7 +832,7 @@ const AssetGraphExplorerWithData = ({ setShowSidebar(false); }} onFilterToGroup={onFilterToGroup} - loading={filteredAssetsLoading} + loading={loading} /> } second={explorer} diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.ts b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.ts new file mode 100644 index 0000000000000..2de03720cdfbf --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.ts @@ -0,0 +1,79 @@ +import groupBy from 'lodash/groupBy'; + +import {ComputeGraphDataMessageType} from './ComputeGraphData.types'; +import {GraphData, buildGraphData, toGraphId} from './Utils'; +import {AssetNodeForGraphQueryFragment} from './types/useAssetGraphData.types'; +import {GraphDataState} from './useAssetGraphData'; +import {filterAssetSelectionByQuery} from '../asset-selection/AntlrAssetSelection'; +import {doesFilterArrayMatchValueArray} from '../ui/Filters/doesFilterArrayMatchValueArray'; + +export function computeGraphData({ + repoFilteredNodes, + graphQueryItems, + opsQuery, + kinds: _kinds, + hideEdgesToNodesOutsideQuery, +}: Omit): GraphDataState { + if (repoFilteredNodes === undefined || graphQueryItems === undefined) { + return { + allAssetKeys: [], + graphAssetKeys: [], + assetGraphData: null, + }; + } + + // Filter the set of all AssetNodes down to those matching the `opsQuery`. + // In the future it might be ideal to move this server-side, but we currently + // get to leverage the useQuery cache almost 100% of the time above, making this + // super fast after the first load vs a network fetch on every page view. + const {all: allFilteredByOpQuery} = filterAssetSelectionByQuery(graphQueryItems, opsQuery); + const kinds = _kinds?.map((c) => c.toLowerCase()); + const all = kinds?.length + ? allFilteredByOpQuery.filter( + ({node}) => + node.kinds && + doesFilterArrayMatchValueArray( + kinds, + node.kinds.map((k) => k.toLowerCase()), + ), + ) + : allFilteredByOpQuery; + + // Assemble the response into the data structure used for layout, traversal, etc. + const assetGraphData = buildGraphData(all.map((n) => n.node)); + if (hideEdgesToNodesOutsideQuery) { + removeEdgesToHiddenAssets(assetGraphData, repoFilteredNodes); + } + + return { + allAssetKeys: repoFilteredNodes.map((n) => n.assetKey), + graphAssetKeys: all.map((n) => ({path: n.node.assetKey.path})), + assetGraphData, + }; +} + +const removeEdgesToHiddenAssets = ( + graphData: GraphData, + allNodes: AssetNodeForGraphQueryFragment[], +) => { + const allNodesById = groupBy(allNodes, (n) => toGraphId(n.assetKey)); + const notSourceAsset = (id: string) => !!allNodesById[id]; + + for (const node of Object.keys(graphData.upstream)) { + for (const edge of Object.keys(graphData.upstream[node]!)) { + if (!graphData.nodes[edge] && notSourceAsset(node)) { + delete graphData.upstream[node]![edge]; + delete graphData.downstream[edge]![node]; + } + } + } + + for (const node of Object.keys(graphData.downstream)) { + for (const edge of Object.keys(graphData.downstream[node]!)) { + if (!graphData.nodes[edge] && notSourceAsset(node)) { + delete graphData.upstream[edge]![node]; + delete graphData.downstream[node]![edge]; + } + } + } +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.types.ts b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.types.ts new file mode 100644 index 0000000000000..e3d0a2a24ea28 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.types.ts @@ -0,0 +1,25 @@ +import {AssetNodeForGraphQueryFragment} from './types/useAssetGraphData.types'; +import {AssetGraphFetchScope, AssetGraphQueryItem} from './useAssetGraphData'; + +type BaseType = { + id: number; + flagAssetSelectionSyntax?: boolean; +}; + +export type ComputeGraphDataMessageType = BaseType & { + type: 'computeGraphData'; + repoFilteredNodes?: AssetNodeForGraphQueryFragment[]; + graphQueryItems?: AssetGraphQueryItem[]; + opsQuery: string; + kinds: AssetGraphFetchScope['kinds']; + hideEdgesToNodesOutsideQuery?: boolean; +}; + +export type BuildGraphDataMessageType = BaseType & { + nodes: AssetNodeForGraphQueryFragment[]; + type: 'buildGraphData'; +}; + +export type ComputeGraphDataWorkerMessageType = + | ComputeGraphDataMessageType + | BuildGraphDataMessageType; diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.worker.ts b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.worker.ts new file mode 100644 index 0000000000000..4a5ea9f754acf --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/ComputeGraphData.worker.ts @@ -0,0 +1,32 @@ +import {FeatureFlag} from 'shared/app/FeatureFlags.oss'; + +import {computeGraphData} from './ComputeGraphData'; +import {BuildGraphDataMessageType, ComputeGraphDataMessageType} from './ComputeGraphData.types'; +import {buildGraphData} from './Utils'; +import {setFeatureFlags} from '../app/Flags'; +import {assertUnreachable} from '../app/Util'; + +type WorkerMessageData = ComputeGraphDataMessageType | BuildGraphDataMessageType; + +self.addEventListener('message', async (event: MessageEvent & {data: WorkerMessageData}) => { + const data: WorkerMessageData = event.data; + + if (data.flagAssetSelectionSyntax) { + setFeatureFlags({[FeatureFlag.flagAssetSelectionSyntax]: true}); + } + + if (data.type === 'computeGraphData') { + const state = await computeGraphData(data); + self.postMessage({...state, id: data.id}); + } else if (data.type === 'buildGraphData') { + self.postMessage({...buildGraphData(data.nodes), id: data.id}); + } else { + assertUnreachable(data); + } +}); + +self.onmessage = function (event) { + if (event.data === 'close') { + self.close(); // Terminates the worker + } +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/Utils.tsx b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/Utils.tsx index 44ac79b147eab..74757d22d1cf8 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/Utils.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/Utils.tsx @@ -132,7 +132,7 @@ export const graphHasCycles = (graphData: GraphData) => { }; let hasCycles = false; while (nodes.size !== 0 && !hasCycles) { - hasCycles = search([], nodes.values().next().value); + hasCycles = search([], nodes.values().next().value!); } return hasCycles; }; diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/__tests__/throttleLatest.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/__tests__/throttleLatest.test.tsx new file mode 100644 index 0000000000000..6a1aaac17fa1f --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/__tests__/throttleLatest.test.tsx @@ -0,0 +1,136 @@ +import {throttleLatest} from '../throttleLatest'; + +jest.useFakeTimers(); + +describe('throttleLatest', () => { + let mockFunction: jest.Mock, [number]>; + let throttledFunction: (arg: number) => Promise; + + beforeEach(() => { + jest.clearAllMocks(); + mockFunction = jest.fn((arg: number) => { + return Promise.resolve(`Result: ${arg}`); + }); + throttledFunction = throttleLatest(mockFunction, 2000); + }); + + it('should execute the first call immediately', async () => { + const promise = throttledFunction(1); + expect(mockFunction).toHaveBeenCalledWith(1); + + await expect(promise).resolves.toBe('Result: 1'); + }); + + it('should throttle subsequent calls within wait time and reject previous promises', async () => { + const promise1 = throttledFunction(1); + const promise2 = throttledFunction(2); + + await expect(promise1).rejects.toThrow('Throttled: A new call has been made.'); + + expect(mockFunction).toHaveBeenCalledTimes(1); + + jest.runAllTimers(); + + await expect(promise2).resolves.toBe('Result: 2'); + }); + + it('should allow a new call after the wait time', async () => { + const promise1 = throttledFunction(1); + + jest.advanceTimersByTime(1000); + + const promise2 = throttledFunction(2); + + await expect(promise1).rejects.toThrow('Throttled: A new call has been made.'); + + jest.advanceTimersByTime(1000); + + await expect(promise2).resolves.toBe('Result: 2'); + + const promise3 = throttledFunction(3); + + await jest.runAllTimers(); + + await expect(promise3).resolves.toBe('Result: 3'); + + expect(mockFunction).toHaveBeenCalledTimes(3); + expect(mockFunction).toHaveBeenNthCalledWith(3, 3); + }); + + it('should handle multiple rapid calls correctly', async () => { + const promise1 = throttledFunction(1); + await Promise.resolve(); + + throttledFunction(2); + + const promise3 = throttledFunction(3); + + await jest.runAllTimers(); + + expect(mockFunction).toHaveBeenNthCalledWith(1, 1); + expect(mockFunction).toHaveBeenCalledTimes(2); + expect(mockFunction).toHaveBeenNthCalledWith(2, 3); + await expect(promise1).resolves.toBe('Result: 1'); + await expect(promise3).resolves.toBe('Result: 3'); + }); + + it('should reject the previous active promise when a new call is made before it resolves', async () => { + // Modify mockFunction to return a promise that doesn't resolve immediately + mockFunction.mockImplementationOnce((arg: number) => { + return new Promise((resolve) => { + setTimeout(() => resolve(`Result: ${arg}`), 5000); + }); + }); + + const promise1 = throttledFunction(1); + + // After 100ms, make a new call + jest.advanceTimersByTime(100); + const promise2 = throttledFunction(2); + + // The first promise should be rejected + await expect(promise1).rejects.toThrow('Throttled: A new call has been made.'); + + // The second promise is scheduled to execute after the remaining time (2000 - 100 = 1900ms) + jest.advanceTimersByTime(1900); + + // Now, the second call should resolve + await expect(promise2).resolves.toBe('Result: 2'); + }); + + it('should handle function rejection correctly', async () => { + mockFunction.mockImplementationOnce(() => { + return Promise.reject(new Error('Function failed')); + }); + + const promise1 = throttledFunction(1); + jest.runAllTimers(); + + await expect(promise1).rejects.toThrow('Function failed'); + }); + + it('should not reject promises if no new call is made within wait time', async () => { + const promise1 = throttledFunction(1); + + // No subsequent calls + jest.runAllTimers(); + + await expect(promise1).resolves.toBe('Result: 1'); + }); + + it('should handle multiple sequential calls with enough time between them', async () => { + const promise1 = throttledFunction(1); + jest.runAllTimers(); + await expect(promise1).resolves.toBe('Result: 1'); + + const promise2 = throttledFunction(2); + jest.runAllTimers(); + await expect(promise2).resolves.toBe('Result: 2'); + + const promise3 = throttledFunction(3); + jest.runAllTimers(); + await expect(promise3).resolves.toBe('Result: 3'); + + expect(mockFunction).toHaveBeenCalledTimes(3); + }); +}); diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/throttleLatest.ts b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/throttleLatest.ts new file mode 100644 index 0000000000000..58ce19fd3b013 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/throttleLatest.ts @@ -0,0 +1,56 @@ +export type ThrottledFunction Promise> = ( + ...args: Parameters +) => ReturnType; + +export function throttleLatest Promise>( + func: T, + wait: number, +): ThrottledFunction { + let timeout: NodeJS.Timeout | null = null; + let lastCallTime: number = 0; + let activeReject: ((reason?: any) => void) | null = null; + + return function (...args: Parameters): ReturnType { + const now = Date.now(); + + return new Promise((resolve, reject) => { + // If a call is already active, reject its promise + if (activeReject) { + activeReject(new Error('Throttled: A new call has been made.')); + activeReject = null; + } + + const execute = () => { + lastCallTime = Date.now(); + activeReject = reject; + + func(...args) + .then((result) => { + resolve(result); + activeReject = null; + }) + .catch((error) => { + reject(error); + activeReject = null; + }); + }; + + const remaining = wait - (now - lastCallTime); + if (remaining <= 0) { + if (timeout) { + clearTimeout(timeout); + timeout = null; + } + execute(); + } else { + if (timeout) { + clearTimeout(timeout); + } + timeout = setTimeout(() => { + execute(); + timeout = null; + }, remaining); + } + }) as ReturnType; + }; +} diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/useAssetGraphData.tsx b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/useAssetGraphData.tsx index b30668c72bc5a..4f08eb1db350e 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/asset-graph/useAssetGraphData.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-graph/useAssetGraphData.tsx @@ -1,11 +1,16 @@ -import groupBy from 'lodash/groupBy'; import keyBy from 'lodash/keyBy'; +import memoize from 'lodash/memoize'; import reject from 'lodash/reject'; -import {useMemo} from 'react'; +import {useEffect, useMemo, useRef, useState} from 'react'; +import {FeatureFlag} from 'shared/app/FeatureFlags.oss'; import {ASSET_NODE_FRAGMENT} from './AssetNode'; -import {GraphData, buildGraphData, toGraphId, tokenForAssetKey} from './Utils'; +import {GraphData, buildGraphData as buildGraphDataImpl, tokenForAssetKey} from './Utils'; import {gql} from '../apollo-client'; +import {computeGraphData as computeGraphDataImpl} from './ComputeGraphData'; +import {BuildGraphDataMessageType, ComputeGraphDataMessageType} from './ComputeGraphData.types'; +import {throttleLatest} from './throttleLatest'; +import {featureEnabled} from '../app/Flags'; import { AssetGraphQuery, AssetGraphQueryVariables, @@ -14,11 +19,11 @@ import { } from './types/useAssetGraphData.types'; import {usePrefixedCacheKey} from '../app/AppProvider'; import {GraphQueryItem} from '../app/GraphQueryImpl'; -import {filterAssetSelectionByQuery} from '../asset-selection/AntlrAssetSelection'; +import {indexedDBAsyncMemoize} from '../app/Util'; import {AssetKey} from '../assets/types'; import {AssetGroupSelector, PipelineSelector} from '../graphql/types'; +import {useBlockTraceUntilTrue} from '../performance/TraceContext'; import {useIndexedDBCachedQuery} from '../search/useIndexedDBCachedQuery'; -import {doesFilterArrayMatchValueArray} from '../ui/Filters/useDefinitionTagFilter'; export interface AssetGraphFetchScope { hideEdgesToNodesOutsideQuery?: boolean; @@ -26,6 +31,10 @@ export interface AssetGraphFetchScope { pipelineSelector?: PipelineSelector; groupSelector?: AssetGroupSelector; kinds?: string[]; + + // This is used to indicate we shouldn't start handling any input. + // This is used by pages where `hideNodesMatching` is only available asynchronously. + loading?: boolean; } export type AssetGraphQueryItem = GraphQueryItem & { @@ -52,15 +61,47 @@ export function useFullAssetGraphData(options: AssetGraphFetchScope) { }); const nodes = fetchResult.data?.assetNodes; - const queryItems = useMemo(() => (nodes ? buildGraphQueryItems(nodes) : []), [nodes]); - - const fullAssetGraphData = useMemo( - () => (queryItems ? buildGraphData(queryItems.map((n) => n.node)) : null), - [queryItems], + const queryItems = useMemo( + () => (nodes ? buildGraphQueryItems(nodes) : []).map(({node}) => node), + [nodes], ); + + const [fullAssetGraphData, setFullAssetGraphData] = useState(null); + useBlockTraceUntilTrue('FullAssetGraphData', !!fullAssetGraphData); + + const lastProcessedRequestRef = useRef(0); + const currentRequestRef = useRef(0); + + useEffect(() => { + if (options.loading) { + return; + } + const requestId = ++currentRequestRef.current; + buildGraphData({ + nodes: queryItems, + flagAssetSelectionSyntax: featureEnabled(FeatureFlag.flagAssetSelectionSyntax), + })?.then((data) => { + if (lastProcessedRequestRef.current < requestId) { + lastProcessedRequestRef.current = requestId; + setFullAssetGraphData(data); + } + }); + }, [options.loading, queryItems]); + return fullAssetGraphData; } +export type GraphDataState = { + graphAssetKeys: AssetKey[]; + allAssetKeys: AssetKey[]; + assetGraphData: GraphData | null; +}; +const INITIAL_STATE: GraphDataState = { + graphAssetKeys: [], + allAssetKeys: [], + assetGraphData: null, +}; + /** Fetches data for rendering an asset graph: * * @param pipelineSelector: Optionally scope to an asset job, or pass null for the global graph @@ -105,58 +146,62 @@ export function useAssetGraphData(opsQuery: string, options: AssetGraphFetchScop [repoFilteredNodes], ); - const {assetGraphData, graphAssetKeys, allAssetKeys} = useMemo(() => { - if (repoFilteredNodes === undefined || graphQueryItems === undefined) { - return { - graphAssetKeys: [], - graphQueryItems: [], - assetGraphData: null, - }; - } + const [state, setState] = useState(INITIAL_STATE); - // Filter the set of all AssetNodes down to those matching the `opsQuery`. - // In the future it might be ideal to move this server-side, but we currently - // get to leverage the useQuery cache almost 100% of the time above, making this - // super fast after the first load vs a network fetch on every page view. - const {all: allFilteredByOpQuery} = filterAssetSelectionByQuery(graphQueryItems, opsQuery); - const kinds = options.kinds?.map((c) => c.toLowerCase()); - const all = kinds?.length - ? allFilteredByOpQuery.filter( - ({node}) => - node.kinds && - doesFilterArrayMatchValueArray( - kinds, - node.kinds.map((k) => k.toLowerCase()), - ), - ) - : allFilteredByOpQuery; - - // Assemble the response into the data structure used for layout, traversal, etc. - const assetGraphData = buildGraphData(all.map((n) => n.node)); - if (options.hideEdgesToNodesOutsideQuery) { - removeEdgesToHiddenAssets(assetGraphData, repoFilteredNodes); - } + const {kinds, hideEdgesToNodesOutsideQuery} = options; + + const [graphDataLoading, setGraphDataLoading] = useState(true); - return { - allAssetKeys: repoFilteredNodes.map((n) => n.assetKey), - graphAssetKeys: all.map((n) => ({path: n.node.assetKey.path})), - assetGraphData, + const lastProcessedRequestRef = useRef(0); + const currentRequestRef = useRef(0); + + useEffect(() => { + if (options.loading) { + return; + } + const requestId = ++currentRequestRef.current; + setGraphDataLoading(true); + computeGraphData({ + repoFilteredNodes, graphQueryItems, - }; + opsQuery, + kinds, + hideEdgesToNodesOutsideQuery, + flagAssetSelectionSyntax: featureEnabled(FeatureFlag.flagAssetSelectionSyntax), + }) + ?.then((data) => { + if (lastProcessedRequestRef.current < requestId) { + lastProcessedRequestRef.current = requestId; + setState(data); + if (requestId === currentRequestRef.current) { + setGraphDataLoading(false); + } + } + }) + .catch((e) => { + console.error(e); + if (requestId === currentRequestRef.current) { + setGraphDataLoading(false); + } + }); }, [ repoFilteredNodes, graphQueryItems, opsQuery, - options.kinds, - options.hideEdgesToNodesOutsideQuery, + kinds, + hideEdgesToNodesOutsideQuery, + options.loading, ]); + const loading = fetchResult.loading || graphDataLoading; + useBlockTraceUntilTrue('useAssetGraphData', !loading); return { + loading, fetchResult, - assetGraphData, + assetGraphData: state.assetGraphData, graphQueryItems, - graphAssetKeys, - allAssetKeys, + graphAssetKeys: state.graphAssetKeys, + allAssetKeys: state.allAssetKeys, }; } @@ -181,29 +226,6 @@ const buildGraphQueryItems = (nodes: AssetNode[]) => { return Object.values(items); }; -const removeEdgesToHiddenAssets = (graphData: GraphData, allNodes: AssetNode[]) => { - const allNodesById = groupBy(allNodes, (n) => toGraphId(n.assetKey)); - const notSourceAsset = (id: string) => !!allNodesById[id]; - - for (const node of Object.keys(graphData.upstream)) { - for (const edge of Object.keys(graphData.upstream[node]!)) { - if (!graphData.nodes[edge] && notSourceAsset(node)) { - delete graphData.upstream[node]![edge]; - delete graphData.downstream[edge]![node]; - } - } - } - - for (const node of Object.keys(graphData.downstream)) { - for (const edge of Object.keys(graphData.downstream[node]!)) { - if (!graphData.nodes[edge] && notSourceAsset(node)) { - delete graphData.upstream[edge]![node]; - delete graphData.downstream[node]![edge]; - } - } - } -}; - export const calculateGraphDistances = (items: GraphQueryItem[], assetKey: AssetKey) => { const map = keyBy(items, (g) => g.name); const start = map[tokenForAssetKey(assetKey)]; @@ -304,3 +326,81 @@ export const ASSET_GRAPH_QUERY = gql` ${ASSET_NODE_FRAGMENT} `; + +const computeGraphData = throttleLatest( + indexedDBAsyncMemoize< + Omit, + GraphDataState, + typeof computeGraphDataWrapper + >(computeGraphDataWrapper, (props) => { + return JSON.stringify(props); + }), + 2000, +); + +const getWorker = memoize( + (_key: string = '') => new Worker(new URL('./ComputeGraphData.worker', import.meta.url)), +); + +let _id = 0; +async function computeGraphDataWrapper( + props: Omit, +): Promise { + if (featureEnabled(FeatureFlag.flagAssetSelectionWorker)) { + const worker = getWorker('computeGraphWorker'); + return new Promise((resolve) => { + const id = ++_id; + const callback = (event: MessageEvent) => { + const data = event.data as GraphDataState & {id: number}; + if (data.id === id) { + resolve(data); + worker.removeEventListener('message', callback); + } + }; + worker.addEventListener('message', callback); + const message: ComputeGraphDataMessageType = { + type: 'computeGraphData', + id, + ...props, + }; + worker.postMessage(message); + }); + } + return computeGraphDataImpl(props); +} + +const buildGraphData = throttleLatest( + indexedDBAsyncMemoize( + buildGraphDataWrapper, + (props) => { + return JSON.stringify(props); + }, + ), + 2000, +); + +async function buildGraphDataWrapper( + props: Omit, +): Promise { + if (featureEnabled(FeatureFlag.flagAssetSelectionWorker)) { + const worker = getWorker('buildGraphWorker'); + return new Promise((resolve) => { + const id = ++_id; + const callback = (event: MessageEvent) => { + const data = event.data as GraphData & {id: number}; + if (data.id === id) { + resolve(data); + worker.removeEventListener('message', callback); + } + }; + worker.addEventListener('message', callback); + const message: BuildGraphDataMessageType = { + type: 'buildGraphData', + id, + ...props, + }; + worker.postMessage(message); + }); + } + return buildGraphDataImpl(props.nodes); +} diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionFiltering.tsx b/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionFiltering.tsx index bcf797b8e5323..564240d2c8eeb 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionFiltering.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionFiltering.tsx @@ -13,9 +13,11 @@ export const useAssetSelectionFiltering = < definition?: FilterableAssetDefinition | null; }, >({ + loading: assetsLoading, assetSelection, assets, }: { + loading?: boolean; assetSelection: string; assets: T[]; @@ -25,7 +27,8 @@ export const useAssetSelectionFiltering = < [assets], ); - const {fetchResult, graphQueryItems, graphAssetKeys} = useAssetGraphData( + const assetsByKeyStringified = useMemo(() => JSON.stringify(assetsByKey), [assetsByKey]); + const {loading, graphQueryItems, graphAssetKeys} = useAssetGraphData( assetSelection, useMemo( () => ({ @@ -33,23 +36,31 @@ export const useAssetSelectionFiltering = < hideNodesMatching: (node: AssetNodeForGraphQueryFragment) => { return !assetsByKey[tokenForAssetKey(node.assetKey)]; }, + loading: !!assetsLoading, }), - [assetsByKey], + // eslint-disable-next-line react-hooks/exhaustive-deps + [assetsByKeyStringified, assetsLoading], ), ); const filtered = useMemo(() => { + if (!assetSelection) { + return assets; + } return ( graphAssetKeys - .map((key) => assetsByKey[tokenForAssetKey(key)]!) + .map((key) => { + return assetsByKey[tokenForAssetKey(key)]!; + }) + .filter((a) => a) .sort((a, b) => COMMON_COLLATOR.compare(a.key.path.join(''), b.key.path.join(''))) ?? [] ); - }, [graphAssetKeys, assetsByKey]); + }, [assetSelection, graphAssetKeys, assets, assetsByKey]); const filteredByKey = useMemo( () => Object.fromEntries(filtered.map((asset) => [tokenForAssetKey(asset.key), asset])), [filtered], ); - return {filtered, filteredByKey, fetchResult, graphAssetKeys, graphQueryItems}; + return {filtered, filteredByKey, loading, graphAssetKeys, graphQueryItems}; }; diff --git a/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionInput.tsx b/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionInput.tsx index 31e5dc83fc717..70bfc494821a5 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionInput.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/asset-selection/useAssetSelectionInput.tsx @@ -15,12 +15,14 @@ export const useAssetSelectionInput = < }, >( assets: T[], + assetsLoading?: boolean, ) => { const [assetSelection, setAssetSelection] = useAssetSelectionState(); - const {graphQueryItems, fetchResult, filtered} = useAssetSelectionFiltering({ + const {graphQueryItems, loading, filtered} = useAssetSelectionFiltering({ assetSelection, assets, + loading: !!assetsLoading, }); let filterInput = ( @@ -43,5 +45,5 @@ export const useAssetSelectionInput = < ); } - return {filterInput, fetchResult, filtered, assetSelection, setAssetSelection}; + return {filterInput, loading, filtered, assetSelection, setAssetSelection}; }; diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/AssetGroupRoot.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/AssetGroupRoot.tsx index 2e4a8fe8ccfc4..51c1544fbc3df 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/assets/AssetGroupRoot.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/assets/AssetGroupRoot.tsx @@ -93,7 +93,7 @@ export const AssetGroupRoot = ({ [history, openInNewTab], ); - const fetchOptions = React.useMemo(() => ({groupSelector}), [groupSelector]); + const fetchOptions = React.useMemo(() => ({groupSelector, loading: false}), [groupSelector]); const lineageOptions = React.useMemo( () => ({preferAssetRendering: true, explodeComposites: true}), diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/AssetsCatalogTable.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/AssetsCatalogTable.tsx index 040f0d3dfe570..5265eaae1d4d6 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/assets/AssetsCatalogTable.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/assets/AssetsCatalogTable.tsx @@ -210,10 +210,10 @@ export const AssetsCatalogTable = ({ activeFiltersJsx, kindFilter, } = useAssetCatalogFiltering({assets}); - const {filterInput, filtered, fetchResult, assetSelection, setAssetSelection} = - useAssetSelectionInput(partiallyFiltered); + const {filterInput, filtered, loading, assetSelection, setAssetSelection} = + useAssetSelectionInput(partiallyFiltered, !assets); - useBlockTraceUntilTrue('useAllAssets', !!assets?.length && !fetchResult.loading); + useBlockTraceUntilTrue('useAllAssets', !!assets?.length && !loading); const {displayPathForAsset, displayed} = useMemo( () => @@ -225,7 +225,7 @@ export const AssetsCatalogTable = ({ const refreshState = useRefreshAtInterval({ refresh: query, - intervalMs: FIFTEEN_SECONDS, + intervalMs: 4 * FIFTEEN_SECONDS, leading: true, }); @@ -255,7 +255,7 @@ export const AssetsCatalogTable = ({ ({})); +jest.mock('../../asset-graph/throttleLatest', () => ({ + throttleLatest: + (fn: any) => + (...args: any[]) => + fn(...args), +})); // These files must be mocked because useVirtualizer tries to create a ResizeObserver, // and the component tree fails to mount. @@ -91,8 +99,10 @@ describe('AssetView', () => { describe('Launch button', () => { it('shows the "Materialize" button for a software-defined asset', async () => { - render(); - expect(await screen.findByText('Materialize')).toBeVisible(); + await act(() => render()); + await waitFor(async () => { + expect(await screen.findByText('Materialize')).toBeVisible(); + }); }); it('shows the "Observe" button for a software-defined source asset', async () => { diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/auto-materialization/GlobalAutomaterializationContent.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/auto-materialization/GlobalAutomaterializationContent.tsx index e019c90eae633..3476d95a3f3a3 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/assets/auto-materialization/GlobalAutomaterializationContent.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/assets/auto-materialization/GlobalAutomaterializationContent.tsx @@ -7,7 +7,7 @@ import { Subtitle2, Table, } from '@dagster-io/ui-components'; -import {useCallback, useMemo, useState} from 'react'; +import {useCallback, useEffect, useMemo, useState} from 'react'; import {ASSET_DAEMON_TICKS_QUERY} from './AssetDaemonTicksQuery'; import {AutomaterializationTickDetailDialog} from './AutomaterializationTickDetailDialog'; @@ -47,6 +47,7 @@ export const GlobalAutomaterializationContent = () => { const [isPaused, setIsPaused] = useState(false); const [statuses, setStatuses] = useState(undefined); const [timeRange, setTimerange] = useState(undefined); + const getVariables = useCallback( (now = Date.now()) => { if (timeRange || statuses) { @@ -72,6 +73,11 @@ export const GlobalAutomaterializationContent = () => { [fetch, getVariables], ); + // When the variables have changed (e.g. due to pagination), refresh. + useEffect(() => { + refresh(); + }, [refresh]); + useRefreshAtInterval({ refresh, enabled: !isPaused && !timeRange && !statuses, diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/useAssetDefinitionFilterState.oss.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/useAssetDefinitionFilterState.oss.tsx index b2681d857761b..c90e38ceb3e17 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/assets/useAssetDefinitionFilterState.oss.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/assets/useAssetDefinitionFilterState.oss.tsx @@ -10,7 +10,8 @@ import { DefinitionTag, } from '../graphql/types'; import {useQueryPersistedState} from '../hooks/useQueryPersistedState'; -import {Tag, doesFilterArrayMatchValueArray} from '../ui/Filters/useDefinitionTagFilter'; +import {doesFilterArrayMatchValueArray} from '../ui/Filters/doesFilterArrayMatchValueArray'; +import {Tag} from '../ui/Filters/useDefinitionTagFilter'; import {buildRepoAddress} from '../workspace/buildRepoAddress'; import {RepoAddress} from '../workspace/types'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/automation/MergedAutomationRoot.tsx b/js_modules/dagster-ui/packages/ui-core/src/automation/MergedAutomationRoot.tsx index 78155c7ff3ca3..b5c1ba848d703 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/automation/MergedAutomationRoot.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/automation/MergedAutomationRoot.tsx @@ -25,9 +25,9 @@ import {makeAutomationKey} from '../sensors/makeSensorKey'; import {useFilters} from '../ui/BaseFilters'; import {useStaticSetFilter} from '../ui/BaseFilters/useStaticSetFilter'; import {CheckAllBox} from '../ui/CheckAllBox'; +import {doesFilterArrayMatchValueArray} from '../ui/Filters/doesFilterArrayMatchValueArray'; import {useCodeLocationFilter} from '../ui/Filters/useCodeLocationFilter'; import { - doesFilterArrayMatchValueArray, useDefinitionTagFilterWithManagedState, useTagsForObjects, } from '../ui/Filters/useDefinitionTagFilter'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx index f979dc3ae59ab..3132b7f265084 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx @@ -138,22 +138,30 @@ export const VirtualizedAutomationScheduleRow = forwardRef( - - {scheduleData ? ( - - {/* Keyed so that a new switch is always rendered, otherwise it's reused and animates on/off */} - - {errorDisplay( - scheduleData.scheduleState.status, - scheduleData.scheduleState.runningCount, - )} - - ) : ( -
- )} - - - + + + {scheduleData ? ( + <> + + {errorDisplay( + scheduleData.scheduleState.status, + scheduleData.scheduleState.runningCount, + )} + + ) : ( +
+ )} + + + + diff --git a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx index 8531c46dabfdc..19a80c0cd2196 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx @@ -136,16 +136,25 @@ export const VirtualizedAutomationSensorRow = forwardRef( - - {/* Keyed so that a new switch is always rendered, otherwise it's reused and animates on/off */} - {sensorData ? ( - - ) : ( -
- )} - - - + + + {/* Keyed so that a new switch is always rendered, otherwise it's reused and animates on/off */} + {sensorData ? ( + + ) : ( +
+ )} + + + + diff --git a/js_modules/dagster-ui/packages/ui-core/src/code-location/CodeLocationOverviewRoot.tsx b/js_modules/dagster-ui/packages/ui-core/src/code-location/CodeLocationOverviewRoot.tsx index 1f998022e2330..8ddff35ccc7e7 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/code-location/CodeLocationOverviewRoot.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/code-location/CodeLocationOverviewRoot.tsx @@ -5,9 +5,9 @@ import { MiddleTruncate, Mono, SpinnerWithText, - StyledRawCodeMirror, Table, } from '@dagster-io/ui-components'; +import {StyledRawCodeMirror} from '@dagster-io/ui-components/editor'; import {useContext, useMemo} from 'react'; import {CodeLocationPageHeader} from 'shared/code-location/CodeLocationPageHeader.oss'; import {CodeLocationServerSection} from 'shared/code-location/CodeLocationServerSection.oss'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/configeditor/ConfigEditorUtils.tsx b/js_modules/dagster-ui/packages/ui-core/src/configeditor/ConfigEditorUtils.tsx index 8e8e256b5013c..81b2e24323da5 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/configeditor/ConfigEditorUtils.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/configeditor/ConfigEditorUtils.tsx @@ -1,4 +1,4 @@ -import {YamlModeValidationResult} from '@dagster-io/ui-components'; +import {YamlModeValidationResult} from '@dagster-io/ui-components/editor'; import yaml from 'yaml'; import {gql} from '../apollo-client'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/graph/OpTags.tsx b/js_modules/dagster-ui/packages/ui-core/src/graph/OpTags.tsx index b9e0032c0f527..00ac437c859c8 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graph/OpTags.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/graph/OpTags.tsx @@ -73,6 +73,7 @@ import hightouch from './kindtag-images/tool-hightouch-color.svg'; import hudi from './kindtag-images/tool-hudi-color.svg'; import huggingface from './kindtag-images/tool-huggingface-color.svg'; import iceberg from './kindtag-images/tool-iceberg-color.svg'; +import icechunk from './kindtag-images/tool-icechunk-color.svg'; import impala from './kindtag-images/tool-impala-color.svg'; import instagram from './kindtag-images/tool-instagram-color.svg'; import java from './kindtag-images/tool-java-color.svg'; @@ -311,6 +312,7 @@ export type KnownTagType = | 'hashicorp' | 'hudi' | 'iceberg' + | 'icechunk' | 'instagram' | 'lakefs' | 'linkedin' @@ -841,6 +843,10 @@ export const KNOWN_TAGS: Record = { icon: iceberg, content: 'Iceberg', }, + icechunk: { + icon: icechunk, + content: 'Icechunk', + }, instagram: { icon: instagram, content: 'Instagram', diff --git a/js_modules/dagster-ui/packages/ui-core/src/graph/asyncGraphLayout.ts b/js_modules/dagster-ui/packages/ui-core/src/graph/asyncGraphLayout.ts index 5ecd65b65199a..f38c20e1d25e2 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graph/asyncGraphLayout.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/graph/asyncGraphLayout.ts @@ -35,8 +35,6 @@ const asyncGetFullOpLayout = asyncMemoize((ops: ILayoutOp[], opts: LayoutOpGraph }); }, _opLayoutCacheKey); -// Asset Graph - const _assetLayoutCacheKey = (graphData: GraphData, opts: LayoutAssetGraphOptions) => { // Note: The "show secondary edges" toggle means that we need a cache key that incorporates // both the displayed nodes and the displayed edges. @@ -202,7 +200,7 @@ export function useAssetLayout( const graphData = useMemo(() => ({..._graphData, expandedGroups}), [expandedGroups, _graphData]); - const cacheKey = _assetLayoutCacheKey(graphData, opts); + const cacheKey = useMemo(() => _assetLayoutCacheKey(graphData, opts), [graphData, opts]); const nodeCount = Object.keys(graphData.nodes).length; const runAsync = nodeCount >= ASYNC_LAYOUT_SOLID_COUNT; diff --git a/js_modules/dagster-ui/packages/ui-core/src/graph/kindtag-images/tool-icechunk-color.svg b/js_modules/dagster-ui/packages/ui-core/src/graph/kindtag-images/tool-icechunk-color.svg new file mode 100644 index 0000000000000..f90c3f3ea7e9d --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/graph/kindtag-images/tool-icechunk-color.svg @@ -0,0 +1,92 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json b/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json index 24d3627e786cf..25046b8064e0f 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json @@ -1 +1 @@ -{"DisplayableEvent":["EngineEvent","ExecutionStepOutputEvent","ExpectationResult","FailureMetadata","HandledOutputEvent","LoadedInputEvent","ObjectStoreOperationResult","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","TypeCheck"],"MarkerEvent":["EngineEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent"],"ErrorEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepUpForRetryEvent","HookErroredEvent","RunCanceledEvent","RunFailureEvent","ResourceInitFailureEvent"],"MessageEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","LogMessageEvent","ObjectStoreOperationEvent","RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetMaterializationPlannedEvent","LogsCapturedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"RunEvent":["RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","AssetMaterializationPlannedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent"],"PipelineRunStepStats":["RunStepStats"],"StepEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","ObjectStoreOperationEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"AssetOwner":["UserAssetOwner","TeamAssetOwner"],"AssetPartitionStatuses":["DefaultPartitionStatuses","MultiPartitionStatuses","TimePartitionStatuses"],"PartitionStatus1D":["TimePartitionStatuses","DefaultPartitionStatuses"],"AssetChecksOrError":["AssetChecks","AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError"],"Instigator":["Schedule","Sensor"],"EvaluationStackEntry":["EvaluationStackListItemEntry","EvaluationStackPathEntry","EvaluationStackMapKeyEntry","EvaluationStackMapValueEntry"],"IPipelineSnapshot":["Pipeline","PipelineSnapshot","Job"],"PipelineConfigValidationError":["FieldNotDefinedConfigError","FieldsNotDefinedConfigError","MissingFieldConfigError","MissingFieldsConfigError","RuntimeMismatchConfigError","SelectorTypeConfigError"],"PipelineConfigValidationInvalid":["RunConfigValidationInvalid"],"PipelineConfigValidationResult":["InvalidSubsetError","PipelineConfigValidationValid","RunConfigValidationInvalid","PipelineNotFoundError","PythonError"],"PipelineReference":["PipelineSnapshot","UnknownPipeline"],"PipelineRun":["Run"],"DagsterRunEvent":["ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","ExecutionStepRestartEvent","LogMessageEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunFailureEvent","RunStartEvent","RunEnqueuedEvent","RunDequeuedEvent","RunStartingEvent","RunCancelingEvent","RunCanceledEvent","RunSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","HandledOutputEvent","LoadedInputEvent","LogsCapturedEvent","ObjectStoreOperationEvent","StepExpectationResultEvent","MaterializationEvent","ObservationEvent","EngineEvent","HookCompletedEvent","HookSkippedEvent","HookErroredEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetMaterializationPlannedEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"PipelineRunLogsSubscriptionPayload":["PipelineRunLogsSubscriptionSuccess","PipelineRunLogsSubscriptionFailure"],"RunOrError":["Run","RunNotFoundError","PythonError"],"PipelineRunStatsSnapshot":["RunStatsSnapshot"],"RunStatsSnapshotOrError":["RunStatsSnapshot","PythonError"],"PipelineSnapshotOrError":["PipelineNotFoundError","PipelineSnapshot","PipelineSnapshotNotFoundError","PythonError"],"RunsFeedEntry":["Run","PartitionBackfill"],"AssetOrError":["Asset","AssetNotFoundError"],"AssetsOrError":["AssetConnection","PythonError"],"DeletePipelineRunResult":["DeletePipelineRunSuccess","UnauthorizedError","PythonError","RunNotFoundError"],"ExecutionPlanOrError":["ExecutionPlan","RunConfigValidationInvalid","PipelineNotFoundError","InvalidSubsetError","PythonError"],"PipelineOrError":["Pipeline","PipelineNotFoundError","InvalidSubsetError","PythonError"],"ReloadRepositoryLocationMutationResult":["WorkspaceLocationEntry","ReloadNotSupported","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"RepositoryLocationOrLoadError":["RepositoryLocation","PythonError"],"ReloadWorkspaceMutationResult":["Workspace","UnauthorizedError","PythonError"],"ShutdownRepositoryLocationMutationResult":["ShutdownRepositoryLocationSuccess","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"TerminatePipelineExecutionFailure":["TerminateRunFailure"],"TerminatePipelineExecutionSuccess":["TerminateRunSuccess"],"TerminateRunResult":["TerminateRunSuccess","TerminateRunFailure","RunNotFoundError","UnauthorizedError","PythonError"],"ScheduleMutationResult":["PythonError","UnauthorizedError","ScheduleStateResult","ScheduleNotFoundError"],"ScheduleOrError":["Schedule","ScheduleNotFoundError","PythonError"],"SchedulerOrError":["Scheduler","SchedulerNotDefinedError","PythonError"],"SchedulesOrError":["Schedules","RepositoryNotFoundError","PythonError"],"ScheduleTickSpecificData":["ScheduleTickSuccessData","ScheduleTickFailureData"],"LaunchBackfillResult":["LaunchBackfillSuccess","PartitionSetNotFoundError","PartitionKeysNotFoundError","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"ConfigTypeOrError":["EnumConfigType","CompositeConfigType","RegularConfigType","PipelineNotFoundError","ConfigTypeNotFoundError","PythonError"],"ConfigType":["ArrayConfigType","CompositeConfigType","EnumConfigType","NullableConfigType","RegularConfigType","ScalarUnionConfigType","MapConfigType"],"WrappingConfigType":["ArrayConfigType","NullableConfigType"],"DagsterType":["ListDagsterType","NullableDagsterType","RegularDagsterType"],"DagsterTypeOrError":["RegularDagsterType","PipelineNotFoundError","DagsterTypeNotFoundError","PythonError"],"WrappingDagsterType":["ListDagsterType","NullableDagsterType"],"Error":["AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError","PartitionKeysNotFoundError","AssetNotFoundError","ConflictingExecutionParamsError","ConfigTypeNotFoundError","DagsterTypeNotFoundError","InvalidPipelineRunsFilterError","InvalidSubsetError","ModeNotFoundError","NoModeProvidedError","PartitionSetNotFoundError","PipelineNotFoundError","RunConflict","PipelineSnapshotNotFoundError","PresetNotFoundError","PythonError","ErrorChainLink","UnauthorizedError","ReloadNotSupported","RepositoryLocationNotFound","RepositoryNotFoundError","ResourceNotFoundError","RunGroupNotFoundError","RunNotFoundError","ScheduleNotFoundError","SchedulerNotDefinedError","SensorNotFoundError","UnsupportedOperationError","DuplicateDynamicPartitionError","InstigationStateNotFoundError","SolidStepStatusUnavailableError","GraphNotFoundError","BackfillNotFoundError","PartitionSubsetDeserializationError","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PipelineRunConflict":["RunConflict"],"PipelineRunNotFoundError":["RunNotFoundError"],"RepositoriesOrError":["RepositoryConnection","RepositoryNotFoundError","PythonError"],"RepositoryOrError":["PythonError","Repository","RepositoryNotFoundError"],"WorkspaceLocationEntryOrError":["WorkspaceLocationEntry","PythonError"],"InstigationTypeSpecificData":["SensorData","ScheduleData"],"InstigationStateOrError":["InstigationState","InstigationStateNotFoundError","PythonError"],"InstigationStatesOrError":["InstigationStates","PythonError"],"MetadataEntry":["TableColumnLineageMetadataEntry","TableSchemaMetadataEntry","TableMetadataEntry","FloatMetadataEntry","IntMetadataEntry","JsonMetadataEntry","BoolMetadataEntry","MarkdownMetadataEntry","PathMetadataEntry","NotebookMetadataEntry","PythonArtifactMetadataEntry","TextMetadataEntry","UrlMetadataEntry","PipelineRunMetadataEntry","AssetMetadataEntry","JobMetadataEntry","CodeReferencesMetadataEntry","NullMetadataEntry","TimestampMetadataEntry"],"SourceLocation":["LocalFileCodeReference","UrlCodeReference"],"PartitionRunConfigOrError":["PartitionRunConfig","PythonError"],"AssetBackfillStatus":["AssetPartitionsStatusCounts","UnpartitionedAssetStatus"],"PartitionSetOrError":["PartitionSet","PartitionSetNotFoundError","PythonError"],"PartitionSetsOrError":["PartitionSets","PipelineNotFoundError","PythonError"],"PartitionsOrError":["Partitions","PythonError"],"PartitionStatusesOrError":["PartitionStatuses","PythonError"],"PartitionTagsOrError":["PartitionTags","PythonError"],"RunConfigSchemaOrError":["RunConfigSchema","PipelineNotFoundError","InvalidSubsetError","ModeNotFoundError","PythonError"],"LaunchRunResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchRunReexecutionResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchPipelineRunSuccess":["LaunchRunSuccess"],"RunsOrError":["Runs","InvalidPipelineRunsFilterError","PythonError"],"PipelineRuns":["Runs"],"RunGroupOrError":["RunGroup","RunGroupNotFoundError","PythonError"],"SensorOrError":["Sensor","SensorNotFoundError","UnauthorizedError","PythonError"],"SensorsOrError":["Sensors","RepositoryNotFoundError","PythonError"],"StopSensorMutationResultOrError":["StopSensorMutationResult","UnauthorizedError","PythonError"],"ISolidDefinition":["CompositeSolidDefinition","SolidDefinition"],"SolidContainer":["Pipeline","PipelineSnapshot","Job","CompositeSolidDefinition","Graph"],"SolidStepStatsOrError":["SolidStepStatsConnection","SolidStepStatusUnavailableError"],"WorkspaceOrError":["Workspace","PythonError"],"WorkspaceLocationStatusEntriesOrError":["WorkspaceLocationStatusEntries","PythonError"],"ResourcesOrError":["ResourceConnection","PipelineNotFoundError","InvalidSubsetError","PythonError"],"GraphOrError":["Graph","GraphNotFoundError","PythonError"],"ResourceDetailsOrError":["ResourceDetails","ResourceNotFoundError","PythonError"],"ResourceDetailsListOrError":["ResourceDetailsList","RepositoryNotFoundError","PythonError"],"EnvVarWithConsumersOrError":["EnvVarWithConsumersList","PythonError"],"RunsFeedConnectionOrError":["RunsFeedConnection","PythonError"],"RunsFeedCountOrError":["RunsFeedCount","PythonError"],"RunTagKeysOrError":["PythonError","RunTagKeys"],"RunTagsOrError":["PythonError","RunTags"],"RunIdsOrError":["RunIds","InvalidPipelineRunsFilterError","PythonError"],"AssetNodeOrError":["AssetNode","AssetNotFoundError"],"PartitionBackfillOrError":["PartitionBackfill","BackfillNotFoundError","PythonError"],"PartitionBackfillsOrError":["PartitionBackfills","PythonError"],"EventConnectionOrError":["EventConnection","RunNotFoundError","PythonError"],"AutoMaterializeAssetEvaluationRecordsOrError":["AutoMaterializeAssetEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PartitionKeysOrError":["PartitionKeys","PartitionSubsetDeserializationError"],"AutoMaterializeRuleEvaluationData":["TextRuleEvaluationData","ParentMaterializedRuleEvaluationData","WaitingOnKeysRuleEvaluationData"],"AssetConditionEvaluationNode":["UnpartitionedAssetConditionEvaluationNode","PartitionedAssetConditionEvaluationNode","SpecificPartitionAssetConditionEvaluationNode"],"AssetConditionEvaluationRecordsOrError":["AssetConditionEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"SensorDryRunResult":["PythonError","SensorNotFoundError","DryRunInstigationTick"],"ScheduleDryRunResult":["DryRunInstigationTick","PythonError","ScheduleNotFoundError"],"TerminateRunsResultOrError":["TerminateRunsResult","PythonError"],"AssetWipeMutationResult":["AssetNotFoundError","UnauthorizedError","PythonError","UnsupportedOperationError","AssetWipeSuccess"],"ReportRunlessAssetEventsResult":["UnauthorizedError","PythonError","ReportRunlessAssetEventsSuccess"],"ResumeBackfillResult":["ResumeBackfillSuccess","UnauthorizedError","PythonError"],"CancelBackfillResult":["CancelBackfillSuccess","UnauthorizedError","PythonError"],"LogTelemetryMutationResult":["LogTelemetrySuccess","PythonError"],"AddDynamicPartitionResult":["AddDynamicPartitionSuccess","UnauthorizedError","PythonError","DuplicateDynamicPartitionError"],"DeleteDynamicPartitionsResult":["DeleteDynamicPartitionsSuccess","UnauthorizedError","PythonError"]} \ No newline at end of file +{"DisplayableEvent":["EngineEvent","ExecutionStepOutputEvent","ExpectationResult","FailureMetadata","HandledOutputEvent","LoadedInputEvent","ObjectStoreOperationResult","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","TypeCheck"],"MarkerEvent":["EngineEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent"],"ErrorEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepUpForRetryEvent","HookErroredEvent","RunCanceledEvent","RunFailureEvent","ResourceInitFailureEvent"],"MessageEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","LogMessageEvent","ObjectStoreOperationEvent","RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetMaterializationPlannedEvent","LogsCapturedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"RunEvent":["RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","AssetMaterializationPlannedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent"],"PipelineRunStepStats":["RunStepStats"],"StepEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","ObjectStoreOperationEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"AssetOwner":["UserAssetOwner","TeamAssetOwner"],"AssetPartitionStatuses":["DefaultPartitionStatuses","MultiPartitionStatuses","TimePartitionStatuses"],"PartitionStatus1D":["TimePartitionStatuses","DefaultPartitionStatuses"],"AssetChecksOrError":["AssetChecks","AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError"],"Instigator":["Schedule","Sensor"],"EvaluationStackEntry":["EvaluationStackListItemEntry","EvaluationStackPathEntry","EvaluationStackMapKeyEntry","EvaluationStackMapValueEntry"],"IPipelineSnapshot":["Pipeline","PipelineSnapshot","Job"],"PipelineConfigValidationError":["FieldNotDefinedConfigError","FieldsNotDefinedConfigError","MissingFieldConfigError","MissingFieldsConfigError","RuntimeMismatchConfigError","SelectorTypeConfigError"],"PipelineConfigValidationInvalid":["RunConfigValidationInvalid"],"PipelineConfigValidationResult":["InvalidSubsetError","PipelineConfigValidationValid","RunConfigValidationInvalid","PipelineNotFoundError","PythonError"],"PipelineReference":["PipelineSnapshot","UnknownPipeline"],"PipelineRun":["Run"],"DagsterRunEvent":["ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","ExecutionStepRestartEvent","LogMessageEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunFailureEvent","RunStartEvent","RunEnqueuedEvent","RunDequeuedEvent","RunStartingEvent","RunCancelingEvent","RunCanceledEvent","RunSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","HandledOutputEvent","LoadedInputEvent","LogsCapturedEvent","ObjectStoreOperationEvent","StepExpectationResultEvent","MaterializationEvent","ObservationEvent","EngineEvent","HookCompletedEvent","HookSkippedEvent","HookErroredEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetMaterializationPlannedEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"PipelineRunLogsSubscriptionPayload":["PipelineRunLogsSubscriptionSuccess","PipelineRunLogsSubscriptionFailure"],"RunOrError":["Run","RunNotFoundError","PythonError"],"PipelineRunStatsSnapshot":["RunStatsSnapshot"],"RunStatsSnapshotOrError":["RunStatsSnapshot","PythonError"],"PipelineSnapshotOrError":["PipelineNotFoundError","PipelineSnapshot","PipelineSnapshotNotFoundError","PythonError"],"RunsFeedEntry":["Run","PartitionBackfill"],"AssetOrError":["Asset","AssetNotFoundError"],"AssetsOrError":["AssetConnection","PythonError"],"DeletePipelineRunResult":["DeletePipelineRunSuccess","UnauthorizedError","PythonError","RunNotFoundError"],"ExecutionPlanOrError":["ExecutionPlan","RunConfigValidationInvalid","PipelineNotFoundError","InvalidSubsetError","PythonError"],"LaunchMultipleRunsResultOrError":["LaunchMultipleRunsResult","PythonError"],"PipelineOrError":["Pipeline","PipelineNotFoundError","InvalidSubsetError","PythonError"],"ReloadRepositoryLocationMutationResult":["WorkspaceLocationEntry","ReloadNotSupported","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"RepositoryLocationOrLoadError":["RepositoryLocation","PythonError"],"ReloadWorkspaceMutationResult":["Workspace","UnauthorizedError","PythonError"],"ShutdownRepositoryLocationMutationResult":["ShutdownRepositoryLocationSuccess","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"TerminatePipelineExecutionFailure":["TerminateRunFailure"],"TerminatePipelineExecutionSuccess":["TerminateRunSuccess"],"TerminateRunResult":["TerminateRunSuccess","TerminateRunFailure","RunNotFoundError","UnauthorizedError","PythonError"],"ScheduleMutationResult":["PythonError","UnauthorizedError","ScheduleStateResult","ScheduleNotFoundError"],"ScheduleOrError":["Schedule","ScheduleNotFoundError","PythonError"],"SchedulerOrError":["Scheduler","SchedulerNotDefinedError","PythonError"],"SchedulesOrError":["Schedules","RepositoryNotFoundError","PythonError"],"ScheduleTickSpecificData":["ScheduleTickSuccessData","ScheduleTickFailureData"],"LaunchBackfillResult":["LaunchBackfillSuccess","PartitionSetNotFoundError","PartitionKeysNotFoundError","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"ConfigTypeOrError":["EnumConfigType","CompositeConfigType","RegularConfigType","PipelineNotFoundError","ConfigTypeNotFoundError","PythonError"],"ConfigType":["ArrayConfigType","CompositeConfigType","EnumConfigType","NullableConfigType","RegularConfigType","ScalarUnionConfigType","MapConfigType"],"WrappingConfigType":["ArrayConfigType","NullableConfigType"],"DagsterType":["ListDagsterType","NullableDagsterType","RegularDagsterType"],"DagsterTypeOrError":["RegularDagsterType","PipelineNotFoundError","DagsterTypeNotFoundError","PythonError"],"WrappingDagsterType":["ListDagsterType","NullableDagsterType"],"Error":["AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError","PartitionKeysNotFoundError","AssetNotFoundError","ConflictingExecutionParamsError","ConfigTypeNotFoundError","DagsterTypeNotFoundError","InvalidPipelineRunsFilterError","InvalidSubsetError","ModeNotFoundError","NoModeProvidedError","PartitionSetNotFoundError","PipelineNotFoundError","RunConflict","PipelineSnapshotNotFoundError","PresetNotFoundError","PythonError","ErrorChainLink","UnauthorizedError","ReloadNotSupported","RepositoryLocationNotFound","RepositoryNotFoundError","ResourceNotFoundError","RunGroupNotFoundError","RunNotFoundError","ScheduleNotFoundError","SchedulerNotDefinedError","SensorNotFoundError","UnsupportedOperationError","DuplicateDynamicPartitionError","InstigationStateNotFoundError","SolidStepStatusUnavailableError","GraphNotFoundError","BackfillNotFoundError","PartitionSubsetDeserializationError","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PipelineRunConflict":["RunConflict"],"PipelineRunNotFoundError":["RunNotFoundError"],"RepositoriesOrError":["RepositoryConnection","RepositoryNotFoundError","PythonError"],"RepositoryOrError":["PythonError","Repository","RepositoryNotFoundError"],"WorkspaceLocationEntryOrError":["WorkspaceLocationEntry","PythonError"],"InstigationTypeSpecificData":["SensorData","ScheduleData"],"InstigationStateOrError":["InstigationState","InstigationStateNotFoundError","PythonError"],"InstigationStatesOrError":["InstigationStates","PythonError"],"MetadataEntry":["TableColumnLineageMetadataEntry","TableSchemaMetadataEntry","TableMetadataEntry","FloatMetadataEntry","IntMetadataEntry","JsonMetadataEntry","BoolMetadataEntry","MarkdownMetadataEntry","PathMetadataEntry","NotebookMetadataEntry","PythonArtifactMetadataEntry","TextMetadataEntry","UrlMetadataEntry","PipelineRunMetadataEntry","AssetMetadataEntry","JobMetadataEntry","CodeReferencesMetadataEntry","NullMetadataEntry","TimestampMetadataEntry"],"SourceLocation":["LocalFileCodeReference","UrlCodeReference"],"PartitionRunConfigOrError":["PartitionRunConfig","PythonError"],"AssetBackfillStatus":["AssetPartitionsStatusCounts","UnpartitionedAssetStatus"],"PartitionSetOrError":["PartitionSet","PartitionSetNotFoundError","PythonError"],"PartitionSetsOrError":["PartitionSets","PipelineNotFoundError","PythonError"],"PartitionsOrError":["Partitions","PythonError"],"PartitionStatusesOrError":["PartitionStatuses","PythonError"],"PartitionTagsOrError":["PartitionTags","PythonError"],"RunConfigSchemaOrError":["RunConfigSchema","PipelineNotFoundError","InvalidSubsetError","ModeNotFoundError","PythonError"],"LaunchRunResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchRunReexecutionResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchPipelineRunSuccess":["LaunchRunSuccess"],"RunsOrError":["Runs","InvalidPipelineRunsFilterError","PythonError"],"PipelineRuns":["Runs"],"RunGroupOrError":["RunGroup","RunGroupNotFoundError","PythonError"],"SensorOrError":["Sensor","SensorNotFoundError","UnauthorizedError","PythonError"],"SensorsOrError":["Sensors","RepositoryNotFoundError","PythonError"],"StopSensorMutationResultOrError":["StopSensorMutationResult","UnauthorizedError","PythonError"],"ISolidDefinition":["CompositeSolidDefinition","SolidDefinition"],"SolidContainer":["Pipeline","PipelineSnapshot","Job","CompositeSolidDefinition","Graph"],"SolidStepStatsOrError":["SolidStepStatsConnection","SolidStepStatusUnavailableError"],"WorkspaceOrError":["Workspace","PythonError"],"WorkspaceLocationStatusEntriesOrError":["WorkspaceLocationStatusEntries","PythonError"],"ResourcesOrError":["ResourceConnection","PipelineNotFoundError","InvalidSubsetError","PythonError"],"GraphOrError":["Graph","GraphNotFoundError","PythonError"],"ResourceDetailsOrError":["ResourceDetails","ResourceNotFoundError","PythonError"],"ResourceDetailsListOrError":["ResourceDetailsList","RepositoryNotFoundError","PythonError"],"EnvVarWithConsumersOrError":["EnvVarWithConsumersList","PythonError"],"RunsFeedConnectionOrError":["RunsFeedConnection","PythonError"],"RunsFeedCountOrError":["RunsFeedCount","PythonError"],"RunTagKeysOrError":["PythonError","RunTagKeys"],"RunTagsOrError":["PythonError","RunTags"],"RunIdsOrError":["RunIds","InvalidPipelineRunsFilterError","PythonError"],"AssetNodeOrError":["AssetNode","AssetNotFoundError"],"PartitionBackfillOrError":["PartitionBackfill","BackfillNotFoundError","PythonError"],"PartitionBackfillsOrError":["PartitionBackfills","PythonError"],"EventConnectionOrError":["EventConnection","RunNotFoundError","PythonError"],"AutoMaterializeAssetEvaluationRecordsOrError":["AutoMaterializeAssetEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PartitionKeysOrError":["PartitionKeys","PartitionSubsetDeserializationError"],"AutoMaterializeRuleEvaluationData":["TextRuleEvaluationData","ParentMaterializedRuleEvaluationData","WaitingOnKeysRuleEvaluationData"],"AssetConditionEvaluationNode":["UnpartitionedAssetConditionEvaluationNode","PartitionedAssetConditionEvaluationNode","SpecificPartitionAssetConditionEvaluationNode"],"AssetConditionEvaluationRecordsOrError":["AssetConditionEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"SensorDryRunResult":["PythonError","SensorNotFoundError","DryRunInstigationTick"],"ScheduleDryRunResult":["DryRunInstigationTick","PythonError","ScheduleNotFoundError"],"TerminateRunsResultOrError":["TerminateRunsResult","PythonError"],"AssetWipeMutationResult":["AssetNotFoundError","UnauthorizedError","PythonError","UnsupportedOperationError","AssetWipeSuccess"],"ReportRunlessAssetEventsResult":["UnauthorizedError","PythonError","ReportRunlessAssetEventsSuccess"],"ResumeBackfillResult":["ResumeBackfillSuccess","UnauthorizedError","PythonError"],"CancelBackfillResult":["CancelBackfillSuccess","UnauthorizedError","PythonError"],"LogTelemetryMutationResult":["LogTelemetrySuccess","PythonError"],"AddDynamicPartitionResult":["AddDynamicPartitionSuccess","UnauthorizedError","PythonError","DuplicateDynamicPartitionError"],"DeleteDynamicPartitionsResult":["DeleteDynamicPartitionsSuccess","UnauthorizedError","PythonError"]} \ No newline at end of file diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql index 7fee0c080d122..2f4c3193df0c9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql @@ -1511,6 +1511,12 @@ type LaunchRunMutation { Output: LaunchRunResult! } +type LaunchMultipleRunsMutation { + Output: LaunchMultipleRunsResultOrError! +} + +union LaunchMultipleRunsResultOrError = LaunchMultipleRunsResult | PythonError + type LaunchRunReexecutionMutation { Output: LaunchRunReexecutionResult! } @@ -2980,6 +2986,10 @@ union LaunchRunResult = | ConflictingExecutionParamsError | NoModeProvidedError +type LaunchMultipleRunsResult { + launchMultipleRunsResult: [LaunchRunResult!]! +} + union LaunchRunReexecutionResult = | LaunchRunSuccess | InvalidStepError @@ -3704,6 +3714,7 @@ type AutomationConditionEvaluationNode { type Mutation { launchPipelineExecution(executionParams: ExecutionParams!): LaunchRunResult! launchRun(executionParams: ExecutionParams!): LaunchRunResult! + launchMultipleRuns(executionParamsList: [ExecutionParams!]!): LaunchMultipleRunsResultOrError! launchPipelineReexecution( executionParams: ExecutionParams reexecutionParams: ReexecutionParams diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts index dffa563c8dcdd..ed57441d95d09 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts @@ -2206,6 +2206,18 @@ export type LaunchBackfillSuccess = { launchedRunIds: Maybe>>; }; +export type LaunchMultipleRunsMutation = { + __typename: 'LaunchMultipleRunsMutation'; + Output: LaunchMultipleRunsResultOrError; +}; + +export type LaunchMultipleRunsResult = { + __typename: 'LaunchMultipleRunsResult'; + launchMultipleRunsResult: Array; +}; + +export type LaunchMultipleRunsResultOrError = LaunchMultipleRunsResult | PythonError; + export type LaunchPipelineRunSuccess = { run: Run; }; @@ -2614,6 +2626,7 @@ export type Mutation = { deleteRun: DeletePipelineRunResult; freeConcurrencySlots: Scalars['Boolean']['output']; freeConcurrencySlotsForRun: Scalars['Boolean']['output']; + launchMultipleRuns: LaunchMultipleRunsResultOrError; launchPartitionBackfill: LaunchBackfillResult; launchPipelineExecution: LaunchRunResult; launchPipelineReexecution: LaunchRunReexecutionResult; @@ -2681,6 +2694,10 @@ export type MutationFreeConcurrencySlotsForRunArgs = { runId: Scalars['String']['input']; }; +export type MutationLaunchMultipleRunsArgs = { + executionParamsList: Array; +}; + export type MutationLaunchPartitionBackfillArgs = { backfillParams: LaunchBackfillParams; }; @@ -9441,6 +9458,38 @@ export const buildLaunchBackfillSuccess = ( }; }; +export const buildLaunchMultipleRunsMutation = ( + overrides?: Partial, + _relationshipsToOmit: Set = new Set(), +): {__typename: 'LaunchMultipleRunsMutation'} & LaunchMultipleRunsMutation => { + const relationshipsToOmit: Set = new Set(_relationshipsToOmit); + relationshipsToOmit.add('LaunchMultipleRunsMutation'); + return { + __typename: 'LaunchMultipleRunsMutation', + Output: + overrides && overrides.hasOwnProperty('Output') + ? overrides.Output! + : relationshipsToOmit.has('LaunchMultipleRunsResult') + ? ({} as LaunchMultipleRunsResult) + : buildLaunchMultipleRunsResult({}, relationshipsToOmit), + }; +}; + +export const buildLaunchMultipleRunsResult = ( + overrides?: Partial, + _relationshipsToOmit: Set = new Set(), +): {__typename: 'LaunchMultipleRunsResult'} & LaunchMultipleRunsResult => { + const relationshipsToOmit: Set = new Set(_relationshipsToOmit); + relationshipsToOmit.add('LaunchMultipleRunsResult'); + return { + __typename: 'LaunchMultipleRunsResult', + launchMultipleRunsResult: + overrides && overrides.hasOwnProperty('launchMultipleRunsResult') + ? overrides.launchMultipleRunsResult! + : [], + }; +}; + export const buildLaunchPipelineRunSuccess = ( overrides?: Partial, _relationshipsToOmit: Set = new Set(), @@ -10222,6 +10271,12 @@ export const buildMutation = ( overrides && overrides.hasOwnProperty('freeConcurrencySlotsForRun') ? overrides.freeConcurrencySlotsForRun! : false, + launchMultipleRuns: + overrides && overrides.hasOwnProperty('launchMultipleRuns') + ? overrides.launchMultipleRuns! + : relationshipsToOmit.has('LaunchMultipleRunsResult') + ? ({} as LaunchMultipleRunsResult) + : buildLaunchMultipleRunsResult({}, relationshipsToOmit), launchPartitionBackfill: overrides && overrides.hasOwnProperty('launchPartitionBackfill') ? overrides.launchPartitionBackfill! diff --git a/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConcurrency.tsx b/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConcurrency.tsx index f38e3f27114b1..3e72fa1ddc767 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConcurrency.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConcurrency.tsx @@ -17,13 +17,13 @@ import { PageHeader, Popover, Spinner, - StyledRawCodeMirror, Subheading, Table, Tag, TextInput, Tooltip, } from '@dagster-io/ui-components'; +import {StyledRawCodeMirror} from '@dagster-io/ui-components/editor'; import * as React from 'react'; import {Link} from 'react-router-dom'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConfig.tsx b/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConfig.tsx index 237c3ab20ddf2..71fca00ebf2d9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConfig.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/instance/InstanceConfig.tsx @@ -7,9 +7,9 @@ import { Heading, PageHeader, Spinner, - StyledRawCodeMirror, Subheading, } from '@dagster-io/ui-components'; +import {StyledRawCodeMirror} from '@dagster-io/ui-components/editor'; import CodeMirror from 'codemirror'; import {memo, useContext, useMemo} from 'react'; import {createGlobalStyle} from 'styled-components'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadRoot.tsx b/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadRoot.tsx index c5aa0ccb46657..55843af536a40 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadRoot.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadRoot.tsx @@ -1,4 +1,5 @@ -import {CodeMirrorInDialogStyle, Dialog, DialogHeader} from '@dagster-io/ui-components'; +import {Dialog, DialogHeader} from '@dagster-io/ui-components'; +import {CodeMirrorInDialogStyle} from '@dagster-io/ui-components/editor'; import {Redirect, useParams} from 'react-router-dom'; import {LaunchpadAllowedRoot} from './LaunchpadAllowedRoot'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadSession.tsx b/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadSession.tsx index cbb164baf6422..6a64493550537 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadSession.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/launchpad/LaunchpadSession.tsx @@ -5,19 +5,21 @@ import { ButtonLink, Checkbox, Colors, - ConfigEditorHandle, - ConfigEditorHelp, - ConfigEditorHelpContext, Dialog, DialogFooter, Group, Icon, - NewConfigEditor, SplitPanelContainer, SplitPanelContainerHandle, TextInput, - isHelpContextEqual, } from '@dagster-io/ui-components'; +import { + ConfigEditorHandle, + ConfigEditorHelp, + ConfigEditorHelpContext, + NewConfigEditor, + isHelpContextEqual, +} from '@dagster-io/ui-components/editor'; import uniqBy from 'lodash/uniqBy'; import * as React from 'react'; import {LaunchRootExecutionButton} from 'shared/launchpad/LaunchRootExecutionButton.oss'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts new file mode 100644 index 0000000000000..9317a6c74597a --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts @@ -0,0 +1,65 @@ +import {useCallback} from 'react'; +import {useHistory} from 'react-router-dom'; + +import {showLaunchError} from './showLaunchError'; +import {useMutation} from '../apollo-client'; +import {TelemetryAction, useTelemetryAction} from '../app/Telemetry'; +import { + LAUNCH_MULTIPLE_RUNS_MUTATION, + LaunchBehavior, + handleLaunchMultipleResult, +} from '../runs/RunUtils'; +import { + LaunchMultipleRunsMutation, + LaunchMultipleRunsMutationVariables, +} from '../runs/types/RunUtils.types'; + +export function useLaunchMultipleRunsWithTelemetry() { + const [launchMultipleRuns] = useMutation< + LaunchMultipleRunsMutation, + LaunchMultipleRunsMutationVariables + >(LAUNCH_MULTIPLE_RUNS_MUTATION); + + const logTelemetry = useTelemetryAction(); + const history = useHistory(); + + return useCallback( + async (variables: LaunchMultipleRunsMutationVariables, behavior: LaunchBehavior) => { + try { + const executionParamsList = Array.isArray(variables.executionParamsList) + ? variables.executionParamsList + : [variables.executionParamsList]; + const jobNames = executionParamsList.map((params) => params.selector?.jobName); + + if ( + jobNames.length !== executionParamsList.length || + jobNames.includes(undefined) || + jobNames.includes(null) + ) { + throw new Error('Invalid job names'); + } + + const metadata: {[key: string]: string | string[] | null | undefined} = { + jobNames: jobNames.filter((name): name is string => name !== undefined), + opSelection: undefined, + }; + + const result = (await launchMultipleRuns({variables})).data?.launchMultipleRuns; + if (result) { + handleLaunchMultipleResult(result, history, {behavior}); + logTelemetry( + TelemetryAction.LAUNCH_MULTIPLE_RUNS, + metadata as {[key: string]: string | string[] | null | undefined}, + ); + } + + return result; + } catch (error) { + console.error('error', error); + showLaunchError(error as Error); + } + return undefined; + }, + [history, launchMultipleRuns, logTelemetry], + ); +} diff --git a/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataScheduler.tsx b/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataScheduler.tsx new file mode 100644 index 0000000000000..6dc5aa18a6334 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataScheduler.tsx @@ -0,0 +1,36 @@ +import type {LiveDataThread} from './LiveDataThread'; + +/** + * This exists as a separate class to allow Jest to mock it + */ +export class LiveDataScheduler { + private thread: LiveDataThread; + + constructor(thread: LiveDataThread) { + this.thread = thread; + } + private _starting = false; + private _stopping = false; + + public scheduleStartFetchLoop(doStart: () => void) { + if (this._starting) { + return; + } + this._starting = true; + setTimeout(() => { + doStart(); + this._starting = false; + }, 50); + } + + public scheduleStopFetchLoop(doStop: () => void) { + if (this._stopping) { + return; + } + this._stopping = true; + setTimeout(() => { + doStop(); + this._stopping = false; + }, 50); + } +} diff --git a/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThread.tsx b/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThread.tsx index 73534ba1ea17f..cc157974f9fd3 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThread.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThread.tsx @@ -1,3 +1,4 @@ +import {LiveDataScheduler} from './LiveDataScheduler'; import {LiveDataThreadManager} from './LiveDataThreadManager'; import {BATCH_PARALLEL_FETCHES, BATCH_SIZE, threadIDToLimits} from './util'; @@ -19,6 +20,8 @@ export class LiveDataThread { return {}; } + private _scheduler: LiveDataScheduler; + constructor( id: string, manager: LiveDataThreadManager, @@ -33,6 +36,7 @@ export class LiveDataThread { this.listenersCount = {}; this.manager = manager; this.intervals = []; + this._scheduler = new LiveDataScheduler(this); } public setPollRate(pollRate: number) { @@ -53,9 +57,7 @@ export class LiveDataThread { if (this.listenersCount[key] === 0) { delete this.listenersCount[key]; } - if (this.getObservedKeys().length === 0) { - this.stopFetchLoop(); - } + this.stopFetchLoop(false); } public getObservedKeys() { @@ -63,19 +65,25 @@ export class LiveDataThread { } public startFetchLoop() { - if (this.activeFetches !== this.parallelFetches) { - requestAnimationFrame(this._batchedQueryKeys); - } - if (this.intervals.length !== this.parallelFetches) { - this.intervals.push(setInterval(this._batchedQueryKeys, 5000)); - } + this._scheduler.scheduleStartFetchLoop(() => { + if (this.activeFetches !== this.parallelFetches) { + requestAnimationFrame(this._batchedQueryKeys); + } + if (this.intervals.length !== this.parallelFetches) { + this.intervals.push(setInterval(this._batchedQueryKeys, 5000)); + } + }); } - public stopFetchLoop() { - this.intervals.forEach((id) => { - clearInterval(id); + public stopFetchLoop(force: boolean) { + this._scheduler.scheduleStopFetchLoop(() => { + if (force || this.getObservedKeys().length === 0) { + this.intervals.forEach((id) => { + clearInterval(id); + }); + this.intervals = []; + } }); - this.intervals = []; } private _batchedQueryKeys = async () => { diff --git a/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThreadManager.tsx b/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThreadManager.tsx index e7118db851ebf..cfc51e7028c91 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThreadManager.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/live-data-provider/LiveDataThreadManager.tsx @@ -174,7 +174,7 @@ export class LiveDataThreadManager { private pause() { this.isPaused = true; Object.values(this.threads).forEach((thread) => { - thread.stopFetchLoop(); + thread.stopFetchLoop(true); }); } diff --git a/js_modules/dagster-ui/packages/ui-core/src/plugins/sql.tsx b/js_modules/dagster-ui/packages/ui-core/src/plugins/sql.tsx index dca45836ffdd2..a1234d42ba834 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/plugins/sql.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/plugins/sql.tsx @@ -1,4 +1,5 @@ -import {Button, Dialog, DialogFooter, Icon, StyledRawCodeMirror} from '@dagster-io/ui-components'; +import {Button, Dialog, DialogFooter, Icon} from '@dagster-io/ui-components'; +import {StyledRawCodeMirror} from '@dagster-io/ui-components/editor'; import {useEffect, useState} from 'react'; import {IPluginSidebarProps} from '../plugins'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx index 3ad2312952379..7dad84a6c971d 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx @@ -1,15 +1,14 @@ -import { - Box, - Button, - Dialog, - DialogFooter, - StyledRawCodeMirror, - Subheading, -} from '@dagster-io/ui-components'; +import {Box, Button, Dialog, DialogFooter, Icon, Subheading} from '@dagster-io/ui-components'; +import {StyledRawCodeMirror} from '@dagster-io/ui-components/editor'; import styled from 'styled-components'; import {RunTags} from './RunTags'; import {RunTagsFragment} from './types/RunTagsFragment.types'; +import {applyCreateSession, useExecutionSessionStorage} from '../app/ExecutionSessionStorage'; +import {useOpenInNewTab} from '../hooks/useOpenInNewTab'; +import {RunRequestFragment} from '../ticks/types/RunRequestFragment.types'; +import {RepoAddress} from '../workspace/types'; +import {workspacePathFromAddress} from '../workspace/workspacePath'; interface Props { isOpen: boolean; @@ -21,10 +20,15 @@ interface Props { // Optionally provide tags to display them as well. tags?: RunTagsFragment[]; + + // Optionally provide a request to display the "Open in Launchpad" button. + request?: RunRequestFragment; + repoAddress?: RepoAddress; } export const RunConfigDialog = (props: Props) => { - const {isOpen, onClose, copyConfig, runConfigYaml, tags, mode, isJob} = props; + const {isOpen, onClose, copyConfig, runConfigYaml, tags, mode, isJob, request, repoAddress} = + props; const hasTags = !!tags && tags.length > 0; return ( @@ -68,7 +72,20 @@ export const RunConfigDialog = (props: Props) => { - + + ) + } + > @@ -81,6 +98,51 @@ export const RunConfigDialog = (props: Props) => { ); }; +function OpenInLaunchpadButton({ + mode, + request, + jobName, + isJob, + repoAddress, +}: { + request: RunRequestFragment; + jobName?: string; + mode?: string | null; + repoAddress: RepoAddress; + isJob: boolean; +}) { + const openInNewTab = useOpenInNewTab(); + const pipelineName = request.jobName ?? jobName; + const [_, onSave] = useExecutionSessionStorage(repoAddress, pipelineName!); + + return ( + + ); +} + const CodeMirrorContainer = styled.div` flex: 1; overflow: hidden; diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx b/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx index 0c8115d91aa30..6a17b5cfc530c 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx @@ -7,7 +7,11 @@ import {StepSelection} from './StepSelection'; import {TimeElapsed} from './TimeElapsed'; import {RunFragment} from './types/RunFragments.types'; import {RunTableRunFragment} from './types/RunTableRunFragment.types'; -import {LaunchPipelineExecutionMutation, RunTimeFragment} from './types/RunUtils.types'; +import { + LaunchMultipleRunsMutation, + LaunchPipelineExecutionMutation, + RunTimeFragment, +} from './types/RunUtils.types'; import {Mono} from '../../../ui-components/src'; import {gql} from '../apollo-client'; import {showCustomAlert} from '../app/CustomAlertProvider'; @@ -107,7 +111,7 @@ export async function handleLaunchResult( if ('errors' in result) { message += ` Please fix the following errors:\n\n${result.errors - .map((error) => error.message) + .map((error: {message: any}) => error.message) .join('\n\n')}`; } @@ -115,6 +119,91 @@ export async function handleLaunchResult( } } +export async function handleLaunchMultipleResult( + result: void | null | LaunchMultipleRunsMutation['launchMultipleRuns'], + history: History, + options: {behavior: LaunchBehavior; preserveQuerystring?: boolean}, +) { + if (!result) { + showCustomAlert({body: `No data was returned. Did dagster-webserver crash?`}); + return; + } + const successfulRunIds: string[] = []; + const failedRunsErrors: {message: string}[] = []; + + if (result.__typename === 'PythonError') { + // if launch multiple runs errors out, show the PythonError and return + showCustomAlert({ + title: 'Error', + body: , + }); + return; + } else if (result.__typename === 'LaunchMultipleRunsResult') { + // show corresponding toasts + const launchMultipleRunsResult = result.launchMultipleRunsResult; + + for (const individualResult of launchMultipleRunsResult) { + if (individualResult.__typename === 'LaunchRunSuccess') { + successfulRunIds.push(individualResult.run.id); + + const pathname = `/runs/${individualResult.run.id}`; + const search = options.preserveQuerystring ? history.location.search : ''; + const openInSameTab = () => history.push({pathname, search}); + + // using open with multiple runs will spam new tabs + if (options.behavior === 'open') { + openInSameTab(); + } + } else if (individualResult.__typename === 'PythonError') { + failedRunsErrors.push({message: individualResult.message}); + } else { + let message = `Error launching run.`; + if ( + individualResult && + typeof individualResult === 'object' && + 'errors' in individualResult + ) { + const errors = individualResult.errors as {message: string}[]; + message += ` Please fix the following errors:\n\n${errors + .map((error) => error.message) + .join('\n\n')}`; + } + if ( + individualResult && + typeof individualResult === 'object' && + 'message' in individualResult + ) { + message += `\n\n${individualResult.message}`; + } + + failedRunsErrors.push({message}); + } + } + } + document.dispatchEvent(new CustomEvent('run-launched')); + + // link to runs page filtered to run IDs + const params = new URLSearchParams(); + successfulRunIds.forEach((id) => params.append('q[]', `id:${id}`)); + + const queryString = `/runs?${params.toString()}`; + history.push(queryString); + + await showSharedToaster({ + intent: 'success', + message:
Launched {successfulRunIds.length} runs
, + action: { + text: 'View', + href: history.createHref({pathname: queryString}), + }, + }); + + // show list of errors that occurred + if (failedRunsErrors.length > 0) { + showCustomAlert({body: failedRunsErrors.map((e) => e.message).join('\n\n')}); + } +} + function getBaseExecutionMetadata(run: RunFragment | RunTableRunFragment) { const hiddenTagKeys: string[] = [DagsterTag.IsResumeRetry, DagsterTag.StepSelection]; @@ -204,6 +293,65 @@ export const LAUNCH_PIPELINE_EXECUTION_MUTATION = gql` ${PYTHON_ERROR_FRAGMENT} `; +export const LAUNCH_MULTIPLE_RUNS_MUTATION = gql` + mutation LaunchMultipleRuns($executionParamsList: [ExecutionParams!]!) { + launchMultipleRuns(executionParamsList: $executionParamsList) { + __typename + ... on LaunchMultipleRunsResult { + launchMultipleRunsResult { + __typename + ... on InvalidStepError { + invalidStepKey + } + ... on InvalidOutputError { + stepKey + invalidOutputName + } + ... on LaunchRunSuccess { + run { + id + pipeline { + name + } + tags { + key + value + } + status + runConfigYaml + mode + resolvedOpSelection + } + } + ... on ConflictingExecutionParamsError { + message + } + ... on PresetNotFoundError { + preset + message + } + ... on RunConfigValidationInvalid { + pipelineName + errors { + __typename + message + path + reason + } + } + ... on PipelineNotFoundError { + message + pipelineName + } + ...PythonErrorFragment + } + } + ...PythonErrorFragment + } + } + ${PYTHON_ERROR_FRAGMENT} +`; + export const DELETE_MUTATION = gql` mutation Delete($runId: String!) { deletePipelineRun(runId: $runId) { diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts index f9dbe944212ae..0ce96156d07a9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts @@ -42,6 +42,106 @@ export type LaunchPipelineExecutionMutation = { | {__typename: 'UnauthorizedError'}; }; +export type LaunchMultipleRunsMutationVariables = Types.Exact<{ + executionParamsList: Array | Types.ExecutionParams; +}>; + +export type LaunchMultipleRunsMutation = { + __typename: 'Mutation'; + launchMultipleRuns: + | { + __typename: 'LaunchMultipleRunsResult'; + launchMultipleRunsResult: Array< + | {__typename: 'ConflictingExecutionParamsError'; message: string} + | {__typename: 'InvalidOutputError'; stepKey: string; invalidOutputName: string} + | {__typename: 'InvalidStepError'; invalidStepKey: string} + | {__typename: 'InvalidSubsetError'} + | { + __typename: 'LaunchRunSuccess'; + run: { + __typename: 'Run'; + id: string; + status: Types.RunStatus; + runConfigYaml: string; + mode: string; + resolvedOpSelection: Array | null; + pipeline: + | {__typename: 'PipelineSnapshot'; name: string} + | {__typename: 'UnknownPipeline'; name: string}; + tags: Array<{__typename: 'PipelineTag'; key: string; value: string}>; + }; + } + | {__typename: 'NoModeProvidedError'} + | {__typename: 'PipelineNotFoundError'; message: string; pipelineName: string} + | {__typename: 'PresetNotFoundError'; preset: string; message: string} + | { + __typename: 'PythonError'; + message: string; + stack: Array; + errorChain: Array<{ + __typename: 'ErrorChainLink'; + isExplicitLink: boolean; + error: {__typename: 'PythonError'; message: string; stack: Array}; + }>; + } + | { + __typename: 'RunConfigValidationInvalid'; + pipelineName: string; + errors: Array< + | { + __typename: 'FieldNotDefinedConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'FieldsNotDefinedConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'MissingFieldConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'MissingFieldsConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'RuntimeMismatchConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'SelectorTypeConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + >; + } + | {__typename: 'RunConflict'} + | {__typename: 'UnauthorizedError'} + >; + } + | { + __typename: 'PythonError'; + message: string; + stack: Array; + errorChain: Array<{ + __typename: 'ErrorChainLink'; + isExplicitLink: boolean; + error: {__typename: 'PythonError'; message: string; stack: Array}; + }>; + }; +}; + export type DeleteMutationVariables = Types.Exact<{ runId: Types.Scalars['String']['input']; }>; @@ -168,6 +268,8 @@ export type RunTimeFragment = { export const LaunchPipelineExecutionVersion = '292088c4a697aca6be1d3bbc0cfc45d8a13cdb2e75cfedc64b68c6245ea34f89'; +export const LaunchMultipleRunsVersion = 'a56d9efdb35e71e0fd1744dd768129248943bc5b23e717458b82c46829661763'; + export const DeleteVersion = '3c61c79b99122910e754a8863e80dc5ed479a0c23cc1a9d9878d91e603fc0dfe'; export const TerminateVersion = '67acf403eb320a93c9a9aa07f675a1557e0887d499cd5598f1d5ff360afc15c0'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx b/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx index ad4cdf9592c43..224bb62400002 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx @@ -1,6 +1,5 @@ import { Box, - Button, Code, Group, Heading, @@ -8,7 +7,6 @@ import { PageHeader, Tag, } from '@dagster-io/ui-components'; -import {useState} from 'react'; import {Link} from 'react-router-dom'; import styled from 'styled-components'; @@ -23,7 +21,7 @@ import {AutomationTargetList} from '../automation/AutomationTargetList'; import {AutomationAssetSelectionFragment} from '../automation/types/AutomationAssetSelectionFragment.types'; import {InstigationStatus} from '../graphql/types'; import {RepositoryLink} from '../nav/RepositoryLink'; -import {EvaluateScheduleDialog} from '../ticks/EvaluateScheduleDialog'; +import {EvaluateTickButtonSchedule} from '../ticks/EvaluateTickButtonSchedule'; import {TickStatusTag} from '../ticks/TickStatusTag'; import {RepoAddress} from '../workspace/types'; @@ -42,8 +40,6 @@ export const ScheduleDetails = (props: { const latestTick = ticks.length > 0 ? ticks[0] : null; const running = status === InstigationStatus.RUNNING; - const [showTestTickDialog, setShowTestTickDialog] = useState(false); - return ( <> - + } /> - { - setShowTestTickDialog(false); - }} - name={schedule.name} - repoAddress={repoAddress} - jobName={pipelineName} - /> {schedule.description ? ( diff --git a/js_modules/dagster-ui/packages/ui-core/src/schedules/SchedulesNextTicks.tsx b/js_modules/dagster-ui/packages/ui-core/src/schedules/SchedulesNextTicks.tsx index 9f6c89ceb6abd..9b2a14e373016 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/schedules/SchedulesNextTicks.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/schedules/SchedulesNextTicks.tsx @@ -14,10 +14,10 @@ import { NonIdealState, Popover, Spinner, - StyledRawCodeMirror, Subheading, Table, } from '@dagster-io/ui-components'; +import {StyledRawCodeMirror} from '@dagster-io/ui-components/editor'; import qs from 'qs'; import {memo, useEffect, useState} from 'react'; import {Link} from 'react-router-dom'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx b/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx index b51301ac76417..2e2d005dac948 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx @@ -17,6 +17,7 @@ import {EditCursorDialog} from './EditCursorDialog'; import {SensorMonitoredAssets} from './SensorMonitoredAssets'; import {SensorResetButton} from './SensorResetButton'; import {SensorSwitch} from './SensorSwitch'; +import {EvaluateTickButtonSensor} from '../ticks/EvaluateTickButtonSensor'; import {SensorFragment} from './types/SensorFragment.types'; import {usePermissionsForLocation} from '../app/Permissions'; import {QueryRefreshCountdown, QueryRefreshState} from '../app/QueryRefresh'; @@ -25,7 +26,6 @@ import {AutomationAssetSelectionFragment} from '../automation/types/AutomationAs import {InstigationStatus, SensorType} from '../graphql/types'; import {RepositoryLink} from '../nav/RepositoryLink'; import {TimestampDisplay} from '../schedules/TimestampDisplay'; -import {SensorDryRunDialog} from '../ticks/SensorDryRunDialog'; import {TickStatusTag} from '../ticks/TickStatusTag'; import {RepoAddress} from '../workspace/types'; @@ -92,7 +92,6 @@ export const SensorDetails = ({ sensor.sensorState.typeSpecificData.__typename === 'SensorData' && sensor.sensorState.typeSpecificData.lastCursor; - const [showTestTickDialog, setShowTestTickDialog] = useState(false); const running = status === InstigationStatus.RUNNING; return ( @@ -114,33 +113,16 @@ export const SensorDetails = ({ right={ - - - + } /> - { - setShowTestTickDialog(false); - }} - currentCursor={cursor || ''} - name={sensor.name} - repoAddress={repoAddress} - jobName={sensor.targets?.[0]?.pipelineName || ''} - /> {sensor.description ? ( diff --git a/js_modules/dagster-ui/packages/ui-core/src/setupTests.ts b/js_modules/dagster-ui/packages/ui-core/src/setupTests.ts index 1b61dbdb34901..8486deec66a7d 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/setupTests.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/setupTests.ts @@ -99,3 +99,6 @@ class MockBroadcastChannel { } (global as any).BroadcastChannel = MockBroadcastChannel; + +// eslint-disable-next-line @typescript-eslint/no-require-imports +require('fast-text-encoding'); diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx index 492f26f1543f6..b9859d06be68a 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx @@ -1,13 +1,14 @@ -import {Box, Button, Colors, Icon, Table, Tag} from '@dagster-io/ui-components'; +import {Box, Button, Colors, Icon, Table, Tooltip} from '@dagster-io/ui-components'; +import {useState} from 'react'; -import {applyCreateSession, useExecutionSessionStorage} from '../app/ExecutionSessionStorage'; +import {RunConfigDialog} from '../runs/RunConfigDialog'; import {RunRequestFragment} from './types/RunRequestFragment.types'; -import {useOpenInNewTab} from '../hooks/useOpenInNewTab'; +import {showSharedToaster} from '../app/DomUtils'; +import {useCopyToClipboard} from '../app/browser'; import {PipelineReference} from '../pipelines/PipelineReference'; import {testId} from '../testing/testId'; import {useRepository} from '../workspace/WorkspaceContext/util'; import {RepoAddress} from '../workspace/types'; -import {workspacePathFromAddress} from '../workspace/workspacePath'; type Props = { name: string; @@ -20,13 +21,25 @@ type Props = { export const RunRequestTable = ({runRequests, isJob, repoAddress, mode, jobName}: Props) => { const repo = useRepository(repoAddress); + const [selectedRequest, setSelectedRequest] = useState(null); + const [visibleDialog, setVisibleDialog] = useState<'config' | null>(null); + const copy = useCopyToClipboard(); + + const copyConfig = async () => { + copy(selectedRequest?.runConfigYaml || ''); + await showSharedToaster({ + intent: 'success', + icon: 'copy_to_clipboard_done', + message: 'Copied!', + }); + }; const body = ( {runRequests.map((request, index) => { return ( - + - - - {filterTags(request.tags).map(({key, value}) => ( - {`${key}: ${value}`} - ))} - - - - + { + setSelectedRequest(request); + setVisibleDialog('config'); + }} /> ); })} + {selectedRequest && ( + setVisibleDialog(null)} + copyConfig={() => copyConfig()} + mode={mode || null} + runConfigYaml={selectedRequest.runConfigYaml} + tags={selectedRequest.tags} + isJob={isJob} + request={selectedRequest} + repoAddress={repoAddress} + /> + )} ); return ( @@ -63,9 +82,8 @@ export const RunRequestTable = ({runRequests, isJob, repoAddress, mode, jobName} - - - + + {body} @@ -74,55 +92,14 @@ export const RunRequestTable = ({runRequests, isJob, repoAddress, mode, jobName} ); }; -// Filter out tags we already display in other ways -function filterTags(tags: Array<{key: string; value: any}>) { - return tags.filter(({key}) => { - // Exclude the tag that specifies the schedule if this is a schedule name - return !['dagster/schedule_name'].includes(key); - }); -} - -function OpenInLaunchpadButton({ - mode, - request, - jobName, - isJob, - repoAddress, -}: { - request: RunRequestFragment; - jobName?: string; - mode?: string; - repoAddress: RepoAddress; - isJob: boolean; -}) { - const openInNewTab = useOpenInNewTab(); - const pipelineName = request.jobName ?? jobName; - const [_, onSave] = useExecutionSessionStorage(repoAddress, pipelineName!); - +function PreviewButton({request, onClick}: {request: RunRequestFragment; onClick: () => void}) { return ( - + + + + // handle showing results page after clicking Evaluate + if (scheduleExecutionData || scheduleExecutionError) { + return ( + + ); + } + + // loading state for evaluating + if (scheduleDryRunMutationLoading) { + return ( + + +
Evaluating schedule
+
+ ); + } else { + // tick selection page + const timestamps = getScheduleData.scheduleOrError.potentialTickTimestamps.map((ts) => ({ + ts, + label: timestampToString({ + timestamp: {unix: ts}, + locale, + timezone: userTimezone, + timeFormat: { + showTimezone: true, + }, + }), + })); + selectedTimestampRef.current = selectedTimestamp || timestamps[0] || null; + return ( + + Select an evaluation time to simulate + + {timestamps.map((timestamp) => ( + {timestamp.label}} + onClick={() => { + setSelectedTimestamp(timestamp); + setIsTickSelectionOpen(false); + }} + /> + ))} + + } + > +
+ +
+
+
+ Each evaluation of a schedule is called a tick, which is an opportunity for one or more + runs to be launched. Ticks kick off runs, which either materialize a selection of assets + or execute a job. + You can preview the result for a given tick in the next step.
- - - ); +
+ Learn more about + schedules +
+
+ ); + } }, [ - _selectedTimestamp, - containerProps, - data, - isTickSelectionOpen, - jobName, - name, + launching, + getScheduleData, + scheduleExecutionData, + scheduleExecutionError, + scheduleDryRunMutationLoading, repoAddress, - shouldEvaluate, - userTimezone, + name, + jobName, + selectedTimestamp, + isTickSelectionOpen, viewport.width, + containerProps, + userTimezone, ]); - const buttons = useMemo(() => { - if (!shouldEvaluate) { + const leftButtons = useMemo(() => { + if (launching) { + return null; + } + + if (scheduleExecutionData || scheduleExecutionError) { + return ( + + ); + } else { + return null; + } + }, [launching, scheduleExecutionData, scheduleExecutionError]); + + const rightButtons = useMemo(() => { + if (launching) { + return ; + } + + if (scheduleExecutionData || scheduleExecutionError) { + const runRequests = scheduleExecutionData?.evaluationResult?.runRequests; + const numRunRequests = runRequests?.length || 0; + const didSkip = !scheduleExecutionError && numRunRequests === 0; + + if (scheduleExecutionError || didSkip) { + return ( + + + + ); + } else { + return ( + + + + + + + ); + } + } + + if (scheduleDryRunMutationLoading) { + return ( + + + + ); + } else { return ( <> - + ); - } else { - return ; } - }, [onClose, shouldEvaluate]); + }, [ + canLaunchAll, + canSubmitTest, + launching, + onClose, + onLaunchAll, + scheduleExecutionData, + scheduleExecutionError, + submitTest, + scheduleDryRunMutationLoading, + ]); return ( <>
{content}
- {buttons ? {buttons} : null} + + {rightButtons} + ); }; @@ -221,62 +463,35 @@ export const GET_SCHEDULE_QUERY = gql` } `; -const EvaluateScheduleContent = ({ +// FE for showing result of evaluating schedule (error, skipped, or success state) +const EvaluateScheduleResult = ({ repoAddress, name, timestamp, jobName, + scheduleExecutionData, + scheduleExecutionError, }: { repoAddress: RepoAddress; name: string; timestamp: number; jobName: string; + scheduleExecutionData: ScheduleDryRunInstigationTick | null; + scheduleExecutionError: PythonErrorFragment | null; }) => { const { timezone: [userTimezone], } = useContext(TimeContext); - const [scheduleDryRunMutation] = useMutation< - ScheduleDryRunMutation, - ScheduleDryRunMutationVariables - >( - SCHEDULE_DRY_RUN_MUTATION, - useMemo(() => { - const repositorySelector = repoAddressToSelector(repoAddress); - return { - variables: { - selectorData: { - ...repositorySelector, - scheduleName: name, - }, - timestamp, - }, - }; - }, [name, repoAddress, timestamp]), - ); - const [result, setResult] = useState> | null>( - null, - ); - useEffect(() => { - scheduleDryRunMutation().then((result) => { - setResult(() => result); - }); - }, [scheduleDryRunMutation]); - if (!result || !result.data) { - return ( - - - - ); - } - - const evaluationResult = - result?.data?.scheduleDryRun.__typename === 'DryRunInstigationTick' - ? result?.data?.scheduleDryRun.evaluationResult - : null; + const evaluationResult = scheduleExecutionData?.evaluationResult; const innerContent = () => { - const data = result.data; + if (scheduleExecutionError) { + return ; + } + + const data = scheduleExecutionData; + if (!data || !evaluationResult) { return ( - Skip Reason -
{evaluationResult?.skipReason || 'No skip reason was output'}
- + + Requested runs (0) +
+ + + + The schedule function was successfully evaluated but didn't return any + run requests. + + +
+ Skip reason:{' '} + {evaluationResult?.skipReason + ? `"${evaluationResult.skipReason}"` + : 'No skip reason was output'} +
+ + } + /> +
+
+
); } else { return ( - + + Requested runs ({numRunRequests}) + + ); } }; const numRunRequests = evaluationResult?.runRequests?.length; - const error = evaluationResult?.error; + const error = scheduleExecutionError || evaluationResult?.error; return ( @@ -358,6 +598,7 @@ const EvaluateScheduleContent = ({ ); }; + export const SCHEDULE_DRY_RUN_MUTATION = gql` mutation ScheduleDryRunMutation($selectorData: ScheduleSelector!, $timestamp: Float) { scheduleDryRun(selectorData: $selectorData, timestamp: $timestamp) { @@ -404,3 +645,11 @@ const Grid = styled.div` const ScheduleDescriptor = styled.div` padding-bottom: 2px; `; + +const SkipReasonNonIdealStateWrapper = styled.div` + ${NonIdealStateWrapper} { + margin: auto !important; + width: unset !important; + max-width: unset !important; + } +`; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx new file mode 100644 index 0000000000000..4713f65e5aa5d --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx @@ -0,0 +1,41 @@ +import {Box, Button, Icon} from '@dagster-io/ui-components'; +import {useState} from 'react'; + +import {EvaluateScheduleDialog} from './EvaluateScheduleDialog'; +import {RepoAddress} from '../workspace/types'; + +interface EvaluateTickButtonScheduleProps { + name: string; + repoAddress: RepoAddress; + jobName: string; +} + +export const EvaluateTickButtonSchedule = ({ + name, + repoAddress, + jobName, +}: EvaluateTickButtonScheduleProps) => { + const [showTestTickDialog, setShowTestTickDialog] = useState(false); + + return ( + + + { + setShowTestTickDialog(false); + }} + name={name} + repoAddress={repoAddress} + jobName={jobName} + /> + + ); +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx new file mode 100644 index 0000000000000..dddbe42c20c50 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx @@ -0,0 +1,50 @@ +import {Box, Button, Icon, Tooltip} from '@dagster-io/ui-components'; +import {useState} from 'react'; + +import {SensorDryRunDialog} from './SensorDryRunDialog'; +import {SensorType} from '../graphql/types'; +import {RepoAddress} from '../workspace/types'; + +interface EvaluateTickButtonSensorProps { + cursor: string; + name: string; + repoAddress: RepoAddress; + jobName: string; + sensorType: SensorType; +} + +export const EvaluateTickButtonSensor = ({ + cursor, + name, + repoAddress, + jobName, + sensorType, +}: EvaluateTickButtonSensorProps) => { + const [showTestTickDialog, setShowTestTickDialog] = useState(false); + + return ( + + + + + setShowTestTickDialog(false)} + currentCursor={cursor} + name={name} + repoAddress={repoAddress} + jobName={jobName} + /> + + ); +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx index b955dde5a967c..1cd0bda5aad43 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx @@ -9,10 +9,12 @@ import { Group, Icon, NonIdealState, + NonIdealStateWrapper, Spinner, Subheading, Tag, TextInput, + Tooltip, } from '@dagster-io/ui-components'; import {useCallback, useMemo, useState} from 'react'; import styled from 'styled-components'; @@ -31,15 +33,18 @@ import {PYTHON_ERROR_FRAGMENT} from '../app/PythonErrorFragment'; import {PythonErrorInfo} from '../app/PythonErrorInfo'; import {assertUnreachable} from '../app/Util'; import {PythonErrorFragment} from '../app/types/PythonErrorFragment.types'; +import {SensorSelector} from '../graphql/types'; +import {useLaunchMultipleRunsWithTelemetry} from '../launchpad/useLaunchMultipleRunsWithTelemetry'; import {SET_CURSOR_MUTATION} from '../sensors/EditCursorDialog'; import { SetSensorCursorMutation, SetSensorCursorMutationVariables, } from '../sensors/types/EditCursorDialog.types'; import {testId} from '../testing/testId'; +import {buildExecutionParamsListSensor} from '../util/buildExecutionParamsList'; import {RepoAddress} from '../workspace/types'; -type DryRunInstigationTick = Extract< +export type SensorDryRunInstigationTick = Extract< SensorDryRunMutation['sensorDryRun'], {__typename: 'DryRunInstigationTick'} >; @@ -60,8 +65,8 @@ export const SensorDryRunDialog = (props: Props) => { isOpen={isOpen} onClose={onClose} style={{width: '70vw', display: 'flex'}} - icon="sensors" - title={name} + icon="preview_tick" + title={`Preview tick result for ${name}`} > @@ -72,16 +77,20 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop const [sensorDryRun] = useMutation( EVALUATE_SENSOR_MUTATION, ); + const [setCursorMutation] = useMutation< + SetSensorCursorMutation, + SetSensorCursorMutationVariables + >(SET_CURSOR_MUTATION); const [cursor, setCursor] = useState(currentCursor); const [submitting, setSubmitting] = useState(false); + const [launching, setLaunching] = useState(false); const [error, setError] = useState(null); - const [sensorExecutionData, setSensorExecutionData] = useState( - null, - ); + const [sensorExecutionData, setSensorExecutionData] = + useState(null); - const sensorSelector = useMemo( + const sensorSelector: SensorSelector = useMemo( () => ({ sensorName: name, repositoryLocationName: repoAddress.location, @@ -90,6 +99,14 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop [repoAddress, name], ); + const executionParamsList = useMemo( + () => + sensorExecutionData && sensorSelector + ? buildExecutionParamsListSensor(sensorExecutionData, sensorSelector) + : [], + [sensorSelector, sensorExecutionData], + ); + const submitTest = useCallback(async () => { setSubmitting(true); const result = await sensorDryRun({ @@ -120,63 +137,16 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop setSubmitting(false); }, [sensorDryRun, sensorSelector, cursor, name]); - const buttons = useMemo(() => { - if (sensorExecutionData || error) { - return ( - - - - - ); - } - if (submitting) { - return ( - - - - ); - } else { - return ( - - - - - ); - } - }, [sensorExecutionData, error, submitting, onClose, submitTest]); - - const [cursorState, setCursorState] = useState<'Unpersisted' | 'Persisting' | 'Persisted'>( - 'Unpersisted', - ); - const [setCursorMutation] = useMutation< - SetSensorCursorMutation, - SetSensorCursorMutationVariables - >(SET_CURSOR_MUTATION); - - const onPersistCursorValue = useCallback(async () => { + const onCommitTickResult = useCallback(async () => { const cursor = sensorExecutionData?.evaluationResult?.cursor; if (!cursor) { - assertUnreachable('Did not expect to get here' as never); + return; } - setCursorState('Persisting'); const {data} = await setCursorMutation({ variables: {sensorSelector, cursor}, }); if (data?.setSensorCursor.__typename === 'Sensor') { await showSharedToaster({message: 'Cursor value updated', intent: 'success'}); - setCursorState('Persisted'); } else if (data?.setSensorCursor) { const error = data.setSensorCursor; await showSharedToaster({ @@ -207,7 +177,153 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop } }, [sensorExecutionData?.evaluationResult?.cursor, sensorSelector, setCursorMutation]); + const launchMultipleRunsWithTelemetry = useLaunchMultipleRunsWithTelemetry(); + + const canLaunchAll = useMemo(() => { + return executionParamsList != null && executionParamsList.length > 0; + }, [executionParamsList]); + + const onLaunchAll = useCallback(async () => { + if (!canLaunchAll) { + return; + } + setLaunching(true); + + try { + if (executionParamsList) { + await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast'); + onCommitTickResult(); // persist tick + } + } catch (e) { + console.error(e); + } + + setLaunching(false); + onClose(); + }, [ + canLaunchAll, + executionParamsList, + launchMultipleRunsWithTelemetry, + onClose, + onCommitTickResult, + ]); + + const leftButtons = useMemo(() => { + if (launching) { + return null; + } + + if (sensorExecutionData || error) { + return ( + + ); + } else { + return null; + } + }, [launching, sensorExecutionData, error]); + + const rightButtons = useMemo(() => { + if (launching) { + return ; + } + + if (sensorExecutionData || error) { + const runRequests = sensorExecutionData?.evaluationResult?.runRequests; + const numRunRequests = runRequests?.length || 0; + const didSkip = !error && numRunRequests === 0; + + if (error) { + return ( + + + + ); + } else if (didSkip) { + return ( + + + + + + + + ); + } else { + return ( + + + + + + + ); + } + } + if (submitting) { + return ( + + + + ); + } else { + return ( + + + + + ); + } + }, [ + launching, + sensorExecutionData, + error, + submitting, + onClose, + onCommitTickResult, + canLaunchAll, + onLaunchAll, + submitTest, + ]); + const content = useMemo(() => { + if (launching) { + return ( + + +
Launching runs
+
+ ); + } if (sensorExecutionData || error) { const runRequests = sensorExecutionData?.evaluationResult?.runRequests; const numRunRequests = runRequests?.length || 0; @@ -216,80 +332,71 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop sensorExecutionData?.evaluationResult?.dynamicPartitionsRequests; return ( + +
+ Result + +
+ {error ? ( + Failed + ) : numRunRequests ? ( + {numRunRequests} run requests + ) : ( + Skipped + )} +
+
+
+
+ Used cursor value +
{cursor?.length ? cursor : 'None'}
+
+
- -
- Result - -
- {error ? ( - Failed - ) : numRunRequests ? ( - {numRunRequests} run requests - ) : ( - Skipped - )} -
-
-
-
- Used cursor value -
{cursor?.length ? cursor : 'None'}
-
-
- Computed cursor value -
-                  {sensorExecutionData?.evaluationResult?.cursor?.length
-                    ? sensorExecutionData?.evaluationResult.cursor
-                    : error
-                      ? 'Error'
-                      : 'None'}
-                
- {error || - (currentCursor ?? '') === - (sensorExecutionData?.evaluationResult?.cursor ?? '') ? null : ( - - - {cursorState === 'Persisted' ? ( - - ) : null} - - )} -
-
{error ? (
) : null} {didSkip ? ( -
- Skip reason + + Requested runs (0)
- {sensorExecutionData?.evaluationResult?.skipReason || 'No skip reason was output'} + + + + The sensor function was successfully evaluated but didn't return + any run requests. + + +
+ Skip reason:{' '} + {sensorExecutionData?.evaluationResult?.skipReason + ? `"${sensorExecutionData.evaluationResult.skipReason}"` + : 'No skip reason was output'} +
+ + } + /> +
-
+
) : null} {numRunRequests && runRequests ? ( - + + Requested runs ({numRunRequests}) + + ) : null} {dynamicPartitionRequests?.length ? (
@@ -297,6 +404,17 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop
) : null}
+ + + Computed cursor value +
+              {sensorExecutionData?.evaluationResult?.cursor?.length
+                ? sensorExecutionData?.evaluationResult.cursor
+                : error
+                  ? 'Error'
+                  : 'None'}
+            
+
); } @@ -310,51 +428,37 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop } else { return ( -
Cursor
+
Cursor value (optional)
setCursor(e.target.value)} data-testid={testId('cursor-input')} + placeholder="Enter a cursor value" /> - {currentCursor === '' || !currentCursor ? ( - - - Check our{' '} - - sensor documentation - {' '} - to learn how to use cursors - - } - /> - - ) : null} +
+ A cursor tracks where a sensor left off, allowing the sensor to efficiently process new + changes or events without missing anything or duplicating work. The cursor is typically + a string, and can be updated within the sensor's logic to reflect the latest state. +
+
+ + Learn more + {' '} + about cursors +
); } - }, [ - sensorExecutionData, - error, - submitting, - currentCursor, - cursorState, - onPersistCursorValue, - name, - jobName, - repoAddress, - cursor, - ]); + }, [sensorExecutionData, error, submitting, launching, name, jobName, repoAddress, cursor]); return ( <>
{content}
- {buttons} + + {rightButtons} + ); }; @@ -394,10 +498,9 @@ export const EVALUATE_SENSOR_MUTATION = gql` const Grid = styled.div` display: grid; - grid-template-columns: repeat(3, 1fr); + grid-template-columns: repeat(2, 1fr); padding-bottom: 12px; border-bottom: 1px solid ${Colors.keylineDefault()}; - margin-bottom: 12px; ${Subheading} { padding-bottom: 4px; display: block; @@ -409,3 +512,27 @@ const Grid = styled.div` margin-top: 4px; } `; + +const ComputedCursorGrid = styled.div` + display: grid; + grid-template-columns: repeat(1, 1fr); + padding-bottom: 12px; + ${Subheading} { + padding-bottom: 4px; + display: block; + } + pre { + margin: 0; + } + button { + margin-top: 4px; + } +`; + +const SkipReasonNonIdealStateWrapper = styled.div` + ${NonIdealStateWrapper} { + margin: auto !important; + width: unset !important; + max-width: unset !important; + } +`; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx index a747734e79869..724b62da00125 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx @@ -1,14 +1,21 @@ import {MockedResponse} from '@apollo/client/testing'; import { + RunStatus, buildDryRunInstigationTick, buildErrorChainLink, + buildLaunchMultipleRunsResult, + buildLaunchRunSuccess, + buildPipelineSnapshot, buildPipelineTag, buildPythonError, + buildRun, buildRunRequest, buildSchedule, buildTickEvaluation, } from '../../graphql/types'; +import {LAUNCH_MULTIPLE_RUNS_MUTATION} from '../../runs/RunUtils'; +import {LaunchMultipleRunsMutation} from '../../runs/types/RunUtils.types'; import {GET_SCHEDULE_QUERY, SCHEDULE_DRY_RUN_MUTATION} from '../EvaluateScheduleDialog'; import {GetScheduleQuery, ScheduleDryRunMutation} from '../types/EvaluateScheduleDialog.types'; @@ -63,7 +70,7 @@ export const scheduleDryWithWithRunRequest = { value: 'okay', }), ], - runKey: null, + runKey: 'EvaluateScheduleDialog.test.tsx:1675705668.993122345', }), ], skipReason: null, @@ -164,3 +171,85 @@ export const ScheduleDryRunMutationSkipped: MockedResponse = { + request: { + query: LAUNCH_MULTIPLE_RUNS_MUTATION, + variables: { + executionParamsList: [ + { + runConfigData: 'ops:\n configurable_op:\n config:\n scheduled_date: 2023-01-29', + selector: { + jobName: 'saepe', + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [ + { + key: 'dagster/schedule_name', + value: 'configurable_job_schedule', + }, + { + key: 'date', + value: '2023-01-29', + }, + { + key: 'github_test', + value: 'test', + }, + { + key: 'okay_t2', + value: 'okay', + }, + ], + }, + }, + ], + }, + }, + result: { + data: { + __typename: 'Mutation', + launchMultipleRuns: buildLaunchMultipleRunsResult({ + launchMultipleRunsResult: [ + buildLaunchRunSuccess({ + run: buildRun({ + id: '504b3a77-d6c4-440c-a128-7f59c9d75d59', + pipeline: buildPipelineSnapshot({ + name: 'saepe', + }), + tags: [ + buildPipelineTag({ + key: 'dagster/schedule_name', + value: 'configurable_job_schedule', + }), + buildPipelineTag({ + key: 'date', + value: '2023-01-29', + }), + buildPipelineTag({ + key: 'github_test', + value: 'test', + }), + buildPipelineTag({ + key: 'okay_t2', + value: 'okay', + }), + ], + status: RunStatus.QUEUED, + runConfigYaml: + 'ops:\n configurable_op:\n config:\n scheduled_date: 2023-01-29', + mode: 'default', + resolvedOpSelection: null, + }), + }), + ], + }), + }, + }, +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx index 4959a6cc6dee3..98e691e5086a9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx @@ -3,16 +3,23 @@ import {MockedResponse} from '@apollo/client/testing'; import { InstigationStatus, RunRequest, + RunStatus, buildDryRunInstigationTick, buildErrorChainLink, buildInstigationState, + buildLaunchMultipleRunsResult, + buildLaunchRunSuccess, + buildPipelineSnapshot, buildPipelineTag, buildPythonError, + buildRun, buildRunRequest, buildSensor, buildSensorData, buildTickEvaluation, } from '../../graphql/types'; +import {LAUNCH_MULTIPLE_RUNS_MUTATION} from '../../runs/RunUtils'; +import {LaunchMultipleRunsMutation} from '../../runs/types/RunUtils.types'; import {SET_CURSOR_MUTATION} from '../../sensors/EditCursorDialog'; import {SetSensorCursorMutation} from '../../sensors/types/EditCursorDialog.types'; import {EVALUATE_SENSOR_MUTATION} from '../SensorDryRunDialog'; @@ -177,3 +184,154 @@ export const PersistCursorValueMock: MockedResponse = { }, }, }; + +export const SensorLaunchAllMutation: MockedResponse = { + request: { + query: LAUNCH_MULTIPLE_RUNS_MUTATION, + variables: { + executionParamsList: [ + { + runConfigData: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx', + selector: { + jobName: 'saepe', + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [ + { + key: 'dagster2', + value: 'test', + }, + { + key: 'marco2', + value: 'salazar2', + }, + ], + }, + }, + { + runConfigData: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx', + selector: { + jobName: 'saepe', + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [ + { + key: 'dagster3', + value: 'test', + }, + { + key: 'marco3', + value: 'salazar3', + }, + ], + }, + }, + { + runConfigData: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx', + selector: { + jobName: 'saepe', + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [ + { + key: 'dagster6', + value: 'test', + }, + { + key: 'marco6', + value: 'salazar6', + }, + ], + }, + }, + ], + }, + }, + result: { + data: { + __typename: 'Mutation', + launchMultipleRuns: buildLaunchMultipleRunsResult({ + launchMultipleRunsResult: [ + buildLaunchRunSuccess({ + __typename: 'LaunchRunSuccess', + run: buildRun({ + __typename: 'Run', + id: '504b3a77-d6c4-440c-a128-7f59c9d75d59', + pipeline: buildPipelineSnapshot({ + name: 'saepe', + }), + tags: [ + buildPipelineTag({key: 'dagster2', value: 'test'}), + buildPipelineTag({key: 'marco2', value: 'salazar2'}), + ], + status: RunStatus.QUEUED, + runConfigYaml: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx\n', + mode: 'default', + resolvedOpSelection: null, + }), + }), + buildLaunchRunSuccess({ + __typename: 'LaunchRunSuccess', + run: buildRun({ + __typename: 'Run', + id: '6745cd03-3d89-4fd2-a41f-6b9d9ffdc134', + pipeline: buildPipelineSnapshot({ + name: 'saepe', + }), + tags: [ + buildPipelineTag({key: 'dagster3', value: 'test'}), + buildPipelineTag({key: 'marco3', value: 'salazar3'}), + ], + + status: RunStatus.QUEUED, + runConfigYaml: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx\n', + mode: 'default', + resolvedOpSelection: null, + }), + }), + buildLaunchRunSuccess({ + run: buildRun({ + id: '7ed35f69-42cf-4518-84a4-c97d0551a56b', + pipeline: buildPipelineSnapshot({ + name: 'simple_config_job', + }), + tags: [ + buildPipelineTag({key: 'dagster6', value: 'test'}), + buildPipelineTag({key: 'marco6', value: 'salazar6'}), + ], + status: RunStatus.QUEUED, + runConfigYaml: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx\n', + + mode: 'default', + resolvedOpSelection: null, + }), + }), + ], + }), + }, + }, +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/DryRunRequestTable.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/DryRunRequestTable.test.tsx index f8f2a760ec648..a1985435b0db5 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/DryRunRequestTable.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/DryRunRequestTable.test.tsx @@ -1,4 +1,4 @@ -import {render, screen} from '@testing-library/react'; +import {render, screen, waitFor} from '@testing-library/react'; import {BrowserRouter} from 'react-router-dom'; import {RunRequestTable} from '../DryRunRequestTable'; @@ -9,6 +9,10 @@ jest.mock('../../workspace/WorkspaceContext/util', () => ({ useRepository: jest.fn(() => null), })); +jest.mock('../../runs/RunConfigDialog', () => ({ + RunConfigDialog: () =>
RunConfigDialog
, +})); + function TestComponent() { return ( @@ -31,10 +35,19 @@ describe('RunRequestTableTest', () => { render(); runRequests.forEach((req) => { - req.tags.forEach(({key, value}) => { - expect(screen.getByText(`${key}: ${value}`)).toBeVisible(); - }); expect(screen.getByTestId(req.runKey!)).toBeVisible(); }); }); + + it('renders preview button and opens dialog on click', async () => { + render(); + + const previewButton = screen.getByTestId(`preview-${runRequests[0]!.runKey || ''}`); + expect(previewButton).toBeVisible(); + previewButton.click(); + + await waitFor(() => { + expect(screen.getByText(/RunConfigDialog/i)).toBeVisible(); + }); + }); }); diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx index c7970d1fba44c..6c17208d0628d 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx @@ -1,6 +1,7 @@ import {MockedProvider, MockedResponse} from '@apollo/client/testing'; import {render, screen, waitFor} from '@testing-library/react'; import userEvent from '@testing-library/user-event'; +import {MemoryRouter, useHistory} from 'react-router-dom'; import {Resolvers} from '../../apollo-client'; import {EvaluateScheduleDialog} from '../EvaluateScheduleDialog'; @@ -9,6 +10,7 @@ import { ScheduleDryRunMutationError, ScheduleDryRunMutationRunRequests, ScheduleDryRunMutationSkipped, + ScheduleLaunchAllMutation, } from '../__fixtures__/EvaluateScheduleDialog.fixtures'; // This component is unit tested separately so mocking it out @@ -18,6 +20,12 @@ jest.mock('../DryRunRequestTable', () => { }; }); +// Mocking useHistory +jest.mock('react-router-dom', () => ({ + ...jest.requireActual('react-router-dom'), + useHistory: jest.fn(), +})); + const onCloseMock = jest.fn(); function Test({mocks, resolvers}: {mocks?: MockedResponse[]; resolvers?: Resolvers}) { @@ -46,7 +54,7 @@ describe('EvaluateScheduleTest', () => { expect(screen.getByTestId('tick-5')).toBeVisible(); }); await userEvent.click(screen.getByTestId('tick-5')); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText(/1\s+run request/i)).toBeVisible(); }); @@ -61,7 +69,10 @@ describe('EvaluateScheduleTest', () => { expect(screen.getByTestId('tick-5')).toBeVisible(); }); await userEvent.click(screen.getByTestId('tick-5')); - await userEvent.click(screen.getByTestId('evaluate')); + await waitFor(() => { + expect(screen.getByTestId('continue')).not.toBeDisabled(); + }); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText('Failed')).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); @@ -76,9 +87,69 @@ describe('EvaluateScheduleTest', () => { expect(screen.getByTestId('tick-5')).toBeVisible(); }); await userEvent.click(screen.getByTestId('tick-5')); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); + await waitFor(() => { + expect(screen.getByText('Skipped')).toBeVisible(); + }); + }); + + it('allows you to test again', async () => { + render(); + const selectButton = await screen.findByTestId('tick-selection'); + await userEvent.click(selectButton); + await waitFor(() => { + expect(screen.getByTestId('tick-5')).toBeVisible(); + }); + await userEvent.click(screen.getByTestId('tick-5')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText('Skipped')).toBeVisible(); }); + await userEvent.click(screen.getByTestId('try-again')); + expect(screen.queryByText('Failed')).toBe(null); + expect(screen.queryByText('Skipped')).toBe(null); + }); + + it('launches all runs', async () => { + const pushSpy = jest.fn(); + const createHrefSpy = jest.fn(); + + (useHistory as jest.Mock).mockReturnValue({ + push: pushSpy, + createHref: createHrefSpy, + }); + + render( + + + , + ); + const selectButton = await screen.findByTestId('tick-selection'); + await userEvent.click(selectButton); + await waitFor(() => { + expect(screen.getByTestId('tick-5')).toBeVisible(); + }); + await userEvent.click(screen.getByTestId('tick-5')); + await userEvent.click(screen.getByTestId('continue')); + await waitFor(() => { + expect(screen.getByText(/1\s+run request/i)).toBeVisible(); + expect(screen.getByTestId('launch-all')).not.toBeDisabled(); + }); + + userEvent.click(screen.getByTestId('launch-all')); + + await waitFor(() => { + expect(screen.getByText(/Launching runs/i)).toBeVisible(); + }); + + await waitFor(() => { + expect(pushSpy).toHaveBeenCalled(); + }); }); }); diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx index 5972b015f2f43..c8307def66a00 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx @@ -1,6 +1,7 @@ import {MockedProvider, MockedResponse} from '@apollo/client/testing'; import {render, screen, waitFor} from '@testing-library/react'; import userEvent from '@testing-library/user-event'; +import {MemoryRouter, useHistory} from 'react-router-dom'; import {Resolvers} from '../../apollo-client'; import {SensorDryRunDialog} from '../SensorDryRunDialog'; @@ -13,6 +14,12 @@ jest.mock('../DryRunRequestTable', () => { }; }); +// Mocking useHistory +jest.mock('react-router-dom', () => ({ + ...jest.requireActual('react-router-dom'), + useHistory: jest.fn(), +})); + const onCloseMock = jest.fn(); function Test({mocks, resolvers}: {mocks?: MockedResponse[]; resolvers?: Resolvers}) { @@ -38,52 +45,91 @@ describe('SensorDryRunTest', () => { render(); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText(/3\srun requests/g)).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); expect(screen.queryByText('Failed')).toBe(null); }); - await userEvent.click(screen.getByTestId('persist-cursor')); - expect(screen.getByText('Persisting')).toBeVisible(); - await waitFor(() => { - expect(screen.getByText('Persisted')).toBeVisible(); - }); }); it('renders errors', async () => { render(); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText('Failed')).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); }); }); + it('renders skip reason', async () => { + render(); + const cursorInput = await screen.findByTestId('cursor-input'); + await userEvent.type(cursorInput, 'testing123'); + await userEvent.click(screen.getByTestId('continue')); + await waitFor(() => { + expect(screen.getByText('Skipped')).toBeVisible(); + }); + }); + it('allows you to test again', async () => { render(); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText('Failed')).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); }); - await userEvent.click(screen.getByTestId('test-again')); + await userEvent.click(screen.getByTestId('try-again')); expect(screen.queryByText('Failed')).toBe(null); expect(screen.queryByText('Skipped')).toBe(null); expect(screen.getByTestId('cursor-input')).toBeVisible(); }); - it('renders skip reason', async () => { - render(); + it('launches all runs', async () => { + const pushSpy = jest.fn(); + const createHrefSpy = jest.fn(); + + (useHistory as jest.Mock).mockReturnValue({ + push: pushSpy, + createHref: createHrefSpy, + }); + + render( + + + , + ); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { - expect(screen.getByText('Skipped')).toBeVisible(); + expect(screen.getByText(/3\srun requests/g)).toBeVisible(); + expect(screen.queryByText('Skipped')).toBe(null); + expect(screen.queryByText('Failed')).toBe(null); + }); + + await waitFor(() => { + expect(screen.getByTestId('launch-all')).not.toBeDisabled(); + }); + + userEvent.click(screen.getByTestId('launch-all')); + + await waitFor(() => { + expect(screen.getByText(/Launching runs/i)).toBeVisible(); + }); + + await waitFor(() => { + expect(pushSpy).toHaveBeenCalled(); }); }); }); diff --git a/js_modules/dagster-ui/packages/ui-core/src/ui/Filters/doesFilterArrayMatchValueArray.tsx b/js_modules/dagster-ui/packages/ui-core/src/ui/Filters/doesFilterArrayMatchValueArray.tsx new file mode 100644 index 0000000000000..a206aca75ced0 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/ui/Filters/doesFilterArrayMatchValueArray.tsx @@ -0,0 +1,25 @@ +import isEqual from 'lodash/isEqual'; + +// This function checks if every element in `filterArray` has at least one match in `valueArray` based on the provided `isMatch` comparison function. +// - `filterArray`: The array containing elements to be matched. +// - `valueArray`: The array to search for matches. +// - `isMatch`: A custom comparator function (defaults to deep equality using `lodash/isEqual`). +// Returns `false` if `filterArray` is non-empty and `valueArray` is empty (no matches possible). +// Otherwise, checks if all elements in `filterArray` have at least one corresponding match in `valueArray`. +// Uses `Array.prototype.some()` to verify if any `filterArray` element lacks a match and returns `false` in such cases. +export function doesFilterArrayMatchValueArray( + filterArray: T[], + valueArray: V[], + isMatch: (value1: T, value2: V) => boolean = (val1, val2) => { + return isEqual(val1, val2); + }, +) { + if (filterArray.length && !valueArray.length) { + return false; + } + return !filterArray.some( + (filterTag) => + // If no asset tags match this filter tag return true + !valueArray.find((value) => isMatch(filterTag, value)), + ); +} diff --git a/js_modules/dagster-ui/packages/ui-core/src/ui/Filters/useDefinitionTagFilter.tsx b/js_modules/dagster-ui/packages/ui-core/src/ui/Filters/useDefinitionTagFilter.tsx index adeaa9855bf60..b309bbb526dc9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ui/Filters/useDefinitionTagFilter.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ui/Filters/useDefinitionTagFilter.tsx @@ -1,4 +1,3 @@ -import isEqual from 'lodash/isEqual'; import memoize from 'lodash/memoize'; import {useCallback, useMemo} from 'react'; @@ -94,23 +93,6 @@ export function useTagsForObjects( ); } -export function doesFilterArrayMatchValueArray( - filterArray: T[], - valueArray: V[], - isMatch: (value1: T, value2: V) => boolean = (val1, val2) => { - return isEqual(val1, val2); - }, -) { - if (filterArray.length && !valueArray.length) { - return false; - } - return !filterArray.some( - (filterTag) => - // If no asset tags match this filter tag return true - !valueArray.find((value) => isMatch(filterTag, value)), - ); -} - export const BaseConfig: StaticBaseConfig = { name: 'Tag', icon: 'tag', diff --git a/js_modules/dagster-ui/packages/ui-core/src/util/__tests__/generateObjectHash.test.ts b/js_modules/dagster-ui/packages/ui-core/src/util/__tests__/generateObjectHash.test.ts new file mode 100644 index 0000000000000..29120193bdbb2 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/util/__tests__/generateObjectHash.test.ts @@ -0,0 +1,99 @@ +import {generateObjectHashStream} from '../generateObjectHash'; + +describe('generateObjectHashStream', () => { + test('hashes a simple object correctly', async () => { + const obj1 = {b: 2, a: 1}; + const obj2 = {a: 1, b: 2}; + + const hash1 = await generateObjectHashStream(obj1); + const hash2 = await generateObjectHashStream(obj2); + + expect(hash1).toBe(hash2); // Should be equal since keys are sorted + }); + + test('hashes nested objects and arrays correctly', async () => { + const obj1 = { + user: { + id: 1, + name: 'Alice', + roles: ['admin', 'user'], + }, + active: true, + }; + + const obj2 = { + active: true, + user: { + roles: ['admin', 'user'], + name: 'Alice', + id: 1, + }, + }; + + const hash1 = await generateObjectHashStream(obj1); + const hash2 = await generateObjectHashStream(obj2); + + expect(hash1).toBe(hash2); // Should be equal due to sorted keys + }); + + test('differentiates between different objects', async () => { + const obj1 = {a: [1]}; + const obj2 = {a: [2]}; + const hash1 = await generateObjectHashStream(obj1); + const hash2 = await generateObjectHashStream(obj2); + expect(hash1).not.toBe(hash2); // Should be different + }); + + test('handles arrays correctly', async () => { + const arr1 = [1, 2, 3]; + const arr2 = [1, 2, 3]; + const arr3 = [3, 2, 1]; + + const hash1 = await generateObjectHashStream(arr1); + const hash2 = await generateObjectHashStream(arr2); + const hash3 = await generateObjectHashStream(arr3); + + expect(hash1).toBe(hash2); + expect(hash1).not.toBe(hash3); + }); + + test('handles empty objects and arrays', async () => { + const emptyObj = {}; + const emptyArr: any[] = []; + + const hashObj = await generateObjectHashStream(emptyObj); + const hashArr = await generateObjectHashStream(emptyArr); + + expect(hashObj).not.toEqual(hashArr); + }); + + test('handles nested arrays correctly', async () => { + const obj1 = { + a: [ + [1, 2], + [3, 4], + ], + }; + const obj2 = { + a: [ + [1, 2], + [3, 5], + ], + }; + + const hash1 = await generateObjectHashStream(obj1); + const hash2 = await generateObjectHashStream(obj2); + + expect(hash1).not.toBe(hash2); + }); + + test('handles different property types', async () => { + const obj1 = {a: 1, b: 'text', c: true}; + const obj2 = {a: 1, b: 'text', c: false}; + + const hash1 = await generateObjectHashStream(obj1); + const hash2 = await generateObjectHashStream(obj2); + + expect(hash1).not.toBe(hash2); + }); +}); diff --git a/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts b/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts new file mode 100644 index 0000000000000..d5264bf96a6b6 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts @@ -0,0 +1,96 @@ +import * as yaml from 'yaml'; + +import {showCustomAlert} from '../app/CustomAlertProvider'; +import {ExecutionParams, ScheduleSelector, SensorSelector} from '../graphql/types'; +import {sanitizeConfigYamlString} from '../launchpad/yamlUtils'; +import {ScheduleDryRunInstigationTick} from '../ticks/EvaluateScheduleDialog'; +import {SensorDryRunInstigationTick} from '../ticks/SensorDryRunDialog'; + +const YAML_SYNTAX_INVALID = `The YAML you provided couldn't be parsed. Please fix the syntax errors and try again.`; + +// This helper removes __typename, which prevents tags from being passed back to GraphQL +const onlyKeyAndValue = ({key, value}: {key: string; value: string}) => ({key, value}); + +// adapted from buildExecutionVariables() in LaunchpadSession.tsx +export const buildExecutionParamsListSensor = ( + sensorExecutionData: SensorDryRunInstigationTick, + sensorSelector: SensorSelector, +) => { + if (!sensorExecutionData) { + return []; + } + + const executionParamsList: ExecutionParams[] = []; + + sensorExecutionData?.evaluationResult?.runRequests?.forEach((request) => { + const configYamlOrEmpty = sanitizeConfigYamlString(request.runConfigYaml); + + try { + yaml.parse(configYamlOrEmpty); + } catch { + showCustomAlert({title: 'Invalid YAML', body: YAML_SYNTAX_INVALID}); + return; + } + const {repositoryLocationName, repositoryName} = sensorSelector; + + const executionParams: ExecutionParams = { + runConfigData: configYamlOrEmpty, + selector: { + jobName: request.jobName, // get jobName from runRequest + repositoryLocationName, + repositoryName, + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [...request.tags.map(onlyKeyAndValue)], + }, + }; + executionParamsList.push(executionParams); + }); + return executionParamsList; +}; + +// adapted from buildExecutionVariables() in LaunchpadSession.tsx +export const buildExecutionParamsListSchedule = ( + scheduleExecutionData: ScheduleDryRunInstigationTick, + scheduleSelector: ScheduleSelector, +) => { + if (!scheduleExecutionData) { + return []; + } + + const executionParamsList: ExecutionParams[] = []; + + scheduleExecutionData?.evaluationResult?.runRequests?.forEach((request) => { + const configYamlOrEmpty = sanitizeConfigYamlString(request.runConfigYaml); + + try { + yaml.parse(configYamlOrEmpty); + } catch { + showCustomAlert({title: 'Invalid YAML', body: YAML_SYNTAX_INVALID}); + return; + } + const {repositoryLocationName, repositoryName} = scheduleSelector; + + const executionParams: ExecutionParams = { + runConfigData: configYamlOrEmpty, + selector: { + jobName: request.jobName, // get jobName from runRequest + repositoryLocationName, + repositoryName, + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [...request.tags.map(onlyKeyAndValue)], + }, + }; + executionParamsList.push(executionParams); + }); + return executionParamsList; +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/util/generateObjectHash.ts b/js_modules/dagster-ui/packages/ui-core/src/util/generateObjectHash.ts new file mode 100644 index 0000000000000..a9e3e664cfae1 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/util/generateObjectHash.ts @@ -0,0 +1,103 @@ +import SparkMD5 from 'spark-md5'; + +/** + * Generates a hash for a JSON-serializable object using incremental JSON serialization + * and SparkMD5 for hashing. + * + * @param obj - The JSON-serializable object to hash. + * @param replacer - Optional JSON.stringify replacer function. + * @returns A Promise that resolves to the hexadecimal string representation of the hash. + */ +export function generateObjectHashStream( + obj: any, + replacer?: (key: string, value: any) => any, +): string { + const hash = new SparkMD5.ArrayBuffer(); + const encoder = new TextEncoder(); + + type Frame = { + obj: any; + keys: string[] | number[]; + index: number; + isArray: boolean; + isFirst: boolean; + }; + + const stack: Frame[] = []; + const isRootArray = Array.isArray(obj); + const initialKeys = isRootArray ? Array.from(Array(obj.length).keys()) : Object.keys(obj).sort(); + stack.push({ + obj, + keys: initialKeys, + index: 0, + isArray: isRootArray, + isFirst: true, + }); + + hash.append(encoder.encode(isRootArray ? '[' : '{')); + + while (stack.length > 0) { + const currentFrame = stack[stack.length - 1]!; + + if (currentFrame.index >= currentFrame.keys.length) { + stack.pop(); + hash.append(encoder.encode(currentFrame.isArray ? ']' : '}')); + if (stack.length > 0) { + const parentFrame = stack[stack.length - 1]!; + parentFrame.isFirst = false; + } + continue; + } + + if (!currentFrame.isFirst) { + hash.append(encoder.encode(',')); + } + currentFrame.isFirst = false; + + const key = currentFrame.keys[currentFrame.index]; + currentFrame.index += 1; + + let value: any; + if (currentFrame.isArray) { + value = currentFrame.obj[key as number]; + } else { + value = currentFrame.obj[key as string]; + } + + value = replacer ? replacer(currentFrame.isArray ? '' : String(key), value) : value; + + if (!currentFrame.isArray) { + const serializedKey = JSON.stringify(key) + ':'; + hash.append(encoder.encode(serializedKey)); + } + + if (value && typeof value === 'object') { + if (Array.isArray(value)) { + hash.append(encoder.encode('[')); + const childKeys = Array.from(Array(value.length).keys()); + stack.push({ + obj: value, + keys: childKeys, + index: 0, + isArray: true, + isFirst: true, + }); + } else { + const childKeys = Object.keys(value).sort(); + hash.append(encoder.encode('{')); + stack.push({ + obj: value, + keys: childKeys, + index: 0, + isArray: false, + isFirst: true, + }); + } + } else { + const serializedValue = JSON.stringify(value); + hash.append(encoder.encode(serializedValue)); + } + } + + return hash.end(); +} diff --git a/js_modules/dagster-ui/packages/ui-core/src/workspace/CodeLocationMenu.tsx b/js_modules/dagster-ui/packages/ui-core/src/workspace/CodeLocationMenu.tsx index ac60eb02aa210..78846b9667bb8 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/workspace/CodeLocationMenu.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/workspace/CodeLocationMenu.tsx @@ -6,9 +6,9 @@ import { Menu, MenuItem, Popover, - StyledRawCodeMirror, Table, } from '@dagster-io/ui-components'; +import {StyledRawCodeMirror} from '@dagster-io/ui-components/editor'; import {useMemo, useState} from 'react'; import * as yaml from 'yaml'; diff --git a/js_modules/dagster-ui/packages/ui-core/tsconfig.json b/js_modules/dagster-ui/packages/ui-core/tsconfig.json index 542426ae2e138..91b6df638a3a0 100644 --- a/js_modules/dagster-ui/packages/ui-core/tsconfig.json +++ b/js_modules/dagster-ui/packages/ui-core/tsconfig.json @@ -2,6 +2,7 @@ "compilerOptions": { "paths": { "@dagster-io/ui-components": ["../ui-components/src"], + "@dagster-io/ui-components/editor": ["../ui-components/src/editor"], "shared/*": ["./src/*"] }, "module": "es2022", diff --git a/js_modules/dagster-ui/yarn.lock b/js_modules/dagster-ui/yarn.lock index cc614aadaf3b3..c1c2f921d4b45 100644 --- a/js_modules/dagster-ui/yarn.lock +++ b/js_modules/dagster-ui/yarn.lock @@ -3727,6 +3727,7 @@ __metadata: "@types/color": "npm:^3.0.2" "@types/dagre": "npm:^0.7.42" "@types/faker": "npm:^5.1.7" + "@types/fast-text-encoding": "npm:^1.0.3" "@types/graphql": "npm:^14.5.0" "@types/jest": "npm:^29.5.11" "@types/lodash": "npm:^4.14.145" @@ -3737,6 +3738,7 @@ __metadata: "@types/react-dom": "npm:^18.3.1" "@types/react-router": "npm:^5.1.17" "@types/react-router-dom": "npm:^5.3.3" + "@types/spark-md5": "npm:^3" "@types/testing-library__jest-dom": "npm:^5.14.2" "@types/ws": "npm:^6.0.3" "@typescript-eslint/eslint-plugin": "npm:^8.9.0" @@ -3768,6 +3770,7 @@ __metadata: eslint-plugin-unused-imports: "npm:^4.1.4" fake-indexeddb: "npm:^4.0.2" faker: "npm:5.5.3" + fast-text-encoding: "npm:^1.0.6" fuse.js: "npm:^6.4.2" graphql: "npm:^16.8.1" graphql-codegen-persisted-query-ids: "npm:^0.2.0" @@ -3801,6 +3804,7 @@ __metadata: remark: "npm:^14.0.2" remark-gfm: "npm:3.0.1" resize-observer-polyfill: "npm:^1.5.1" + spark-md5: "npm:^3.0.2" storybook: "npm:^8.2.7" strip-markdown: "npm:^6.0.0" styled-components: "npm:^6" @@ -7164,6 +7168,13 @@ __metadata: languageName: node linkType: hard +"@types/fast-text-encoding@npm:^1.0.3": + version: 1.0.3 + resolution: "@types/fast-text-encoding@npm:1.0.3" + checksum: 10/34ec2bbaf3e3ee36b7b74375293becc735378f77e9cd93b810ad988b42991ee80d30fb942e6ba03adfc1f0cb0e2024a0aeee84475847563ed6782e21c4c0f5f0 + languageName: node + linkType: hard + "@types/graceful-fs@npm:^4.1.2, @types/graceful-fs@npm:^4.1.3": version: 4.1.6 resolution: "@types/graceful-fs@npm:4.1.6" @@ -7565,6 +7576,13 @@ __metadata: languageName: node linkType: hard +"@types/spark-md5@npm:^3": + version: 3.0.5 + resolution: "@types/spark-md5@npm:3.0.5" + checksum: 10/b543313e8669db34259aa67cff281f63b6746e08711e2b93d653cbb32ec63bb6153e75eeb534d3e874b5a6c1cb8cbe099dd85f9f912b23d9b0f4d51f3e968a2e + languageName: node + linkType: hard + "@types/stack-utils@npm:^2.0.0": version: 2.0.1 resolution: "@types/stack-utils@npm:2.0.1" @@ -13221,6 +13239,13 @@ __metadata: languageName: node linkType: hard +"fast-text-encoding@npm:^1.0.6": + version: 1.0.6 + resolution: "fast-text-encoding@npm:1.0.6" + checksum: 10/f7b9e2e7a21e4ae5f4b8d3729850be83fb45052b28c9c38c09b8366463a291d6dc5448359238bdaf87f6a9e907d5895a94319a2c5e0e9f0786859ad6312d1d06 + languageName: node + linkType: hard + "fast-url-parser@npm:^1.1.3": version: 1.1.3 resolution: "fast-url-parser@npm:1.1.3" @@ -17756,21 +17781,12 @@ __metadata: languageName: node linkType: hard -"nanoid@npm:^3.3.6": - version: 3.3.6 - resolution: "nanoid@npm:3.3.6" +"nanoid@npm:^3.3.6, nanoid@npm:^3.3.7": + version: 3.3.8 + resolution: "nanoid@npm:3.3.8" bin: nanoid: bin/nanoid.cjs - checksum: 10/67235c39d1bc05851383dadde5cf77ae1c90c2a1d189e845c7f20f646f0488d875ad5f5226bbba072a88cebbb085a3f784a6673117daf785bdf614a852550362 - languageName: node - linkType: hard - -"nanoid@npm:^3.3.7": - version: 3.3.7 - resolution: "nanoid@npm:3.3.7" - bin: - nanoid: bin/nanoid.cjs - checksum: 10/ac1eb60f615b272bccb0e2b9cd933720dad30bf9708424f691b8113826bb91aca7e9d14ef5d9415a6ba15c266b37817256f58d8ce980c82b0ba3185352565679 + checksum: 10/2d1766606cf0d6f47b6f0fdab91761bb81609b2e3d367027aff45e6ee7006f660fb7e7781f4a34799fe6734f1268eeed2e37a5fdee809ade0c2d4eb11b0f9c40 languageName: node linkType: hard @@ -21482,6 +21498,13 @@ __metadata: languageName: node linkType: hard +"spark-md5@npm:^3.0.2": + version: 3.0.2 + resolution: "spark-md5@npm:3.0.2" + checksum: 10/60981e181a296b2d16064ef86607f78d7eb1e08a5f39366239bb9cdd6bc3838fb2f667f2506e81c8d5c71965cdd6f18a17fb1c9a8368eeb407b9dd8188e95473 + languageName: node + linkType: hard + "split-on-first@npm:^1.0.0": version: 1.1.0 resolution: "split-on-first@npm:1.1.0" diff --git a/python_modules/automation/automation_tests/telemetry/test_resource_telemetry.py b/python_modules/automation/automation_tests/telemetry/test_resource_telemetry.py index 386c40cd52203..3a0154135705f 100644 --- a/python_modules/automation/automation_tests/telemetry/test_resource_telemetry.py +++ b/python_modules/automation/automation_tests/telemetry/test_resource_telemetry.py @@ -37,6 +37,8 @@ def test_resource_telemetry(): libraries.remove("dagster_airflow") # new library, not added yet libraries.remove("dagster_embedded_elt") + # temporary library that will be merged into dagster + libraries.remove("dagster_components") resources_without_telemetry = [] diff --git a/python_modules/dagster-graphql/dagster_graphql/client/query.py b/python_modules/dagster-graphql/dagster_graphql/client/query.py index 579b4e6e7a046..76f5ea6d9607f 100644 --- a/python_modules/dagster-graphql/dagster_graphql/client/query.py +++ b/python_modules/dagster-graphql/dagster_graphql/client/query.py @@ -330,6 +330,71 @@ ) +LAUNCH_MULTIPLE_RUNS_MUTATION = ( + ERROR_FRAGMENT + + """ +mutation($executionParamsList: [ExecutionParams!]!) { + launchMultipleRuns(executionParamsList: $executionParamsList) { + __typename + ... on LaunchMultipleRunsResult { + launchMultipleRunsResult { + __typename + ... on InvalidStepError { + invalidStepKey + } + ... on InvalidOutputError { + stepKey + invalidOutputName + } + ... on LaunchRunSuccess { + run { + runId + pipeline { + name + } + tags { + key + value + } + status + runConfigYaml + mode + resolvedOpSelection + } + } + ... on ConflictingExecutionParamsError { + message + } + ... on PresetNotFoundError { + preset + message + } + ... on RunConfigValidationInvalid { + pipelineName + errors { + __typename + message + path + reason + } + } + ... on PipelineNotFoundError { + message + pipelineName + } + ... on PythonError { + ...errorFragment + } + } + } + ... on PythonError { + ...errorFragment + } + } +} +""" +) + LAUNCH_PIPELINE_REEXECUTION_MUTATION = ( ERROR_FRAGMENT + """ diff --git a/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py b/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py index a15d3bf972a92..c1716bfb25861 100644 --- a/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py +++ b/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py @@ -312,19 +312,14 @@ def cancel_partition_backfill( backfill.asset_selection, Permissions.CANCEL_PARTITION_BACKFILL, ) - graphene_info.context.instance.update_backfill( - backfill.with_status(BulkActionStatus.CANCELING) - ) - else: partition_set_origin = check.not_none(backfill.partition_set_origin) location_name = partition_set_origin.selector.location_name assert_permission_for_location( graphene_info, Permissions.CANCEL_PARTITION_BACKFILL, location_name ) - graphene_info.context.instance.update_backfill( - backfill.with_status(BulkActionStatus.CANCELED) - ) + + graphene_info.context.instance.update_backfill(backfill.with_status(BulkActionStatus.CANCELING)) return GrapheneCancelBackfillSuccess(backfill_id=backfill_id) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py b/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py index 6533d2d0b0f32..10f029145bc11 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py @@ -10,6 +10,7 @@ def types(): GrapheneDeletePipelineRunSuccess, GrapheneDeleteRunMutation, GrapheneLaunchBackfillMutation, + GrapheneLaunchMultipleRunsMutation, GrapheneLaunchRunMutation, GrapheneLaunchRunReexecutionMutation, GrapheneReloadRepositoryLocationMutation, @@ -38,6 +39,7 @@ def types(): GrapheneExecutionPlanOrError, GrapheneLaunchBackfillMutation, GrapheneLaunchRunMutation, + GrapheneLaunchMultipleRunsMutation, GrapheneLaunchRunReexecutionMutation, GraphenePipelineOrError, GrapheneReloadRepositoryLocationMutation, diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py b/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py index 0e98ceda547ee..687d1fd70ba60 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py @@ -1,4 +1,4 @@ -from typing import Optional, Sequence, Union +from typing import List, Optional, Sequence, Union import dagster._check as check import graphene @@ -78,6 +78,8 @@ ) from dagster_graphql.schema.pipelines.pipeline import GrapheneRun from dagster_graphql.schema.runs import ( + GrapheneLaunchMultipleRunsResult, + GrapheneLaunchMultipleRunsResultOrError, GrapheneLaunchRunReexecutionResult, GrapheneLaunchRunResult, GrapheneLaunchRunSuccess, @@ -316,6 +318,38 @@ def mutate( return create_execution_params_and_launch_pipeline_exec(graphene_info, executionParams) +class GrapheneLaunchMultipleRunsMutation(graphene.Mutation): + """Launches multiple job runs.""" + + Output = graphene.NonNull(GrapheneLaunchMultipleRunsResultOrError) + + class Arguments: + executionParamsList = non_null_list(GrapheneExecutionParams) + + class Meta: + name = "LaunchMultipleRunsMutation" + + @capture_error + def mutate( + self, graphene_info: ResolveInfo, executionParamsList: List[GrapheneExecutionParams] + ) -> Union[ + GrapheneLaunchMultipleRunsResult, + GrapheneError, + GraphenePythonError, + ]: + launch_multiple_runs_result = [] + + for execution_params in executionParamsList: + result = GrapheneLaunchRunMutation.mutate( + None, graphene_info, executionParams=execution_params + ) + launch_multiple_runs_result.append(result) + + return GrapheneLaunchMultipleRunsResult( + launchMultipleRunsResult=launch_multiple_runs_result + ) + + class GrapheneLaunchBackfillMutation(graphene.Mutation): """Launches a set of partition backfill runs.""" @@ -984,6 +1018,7 @@ class Meta: launchPipelineExecution = GrapheneLaunchRunMutation.Field() launchRun = GrapheneLaunchRunMutation.Field() + launchMultipleRuns = GrapheneLaunchMultipleRunsMutation.Field() launchPipelineReexecution = GrapheneLaunchRunReexecutionMutation.Field() launchRunReexecution = GrapheneLaunchRunReexecutionMutation.Field() startSchedule = GrapheneStartScheduleMutation.Field() diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py index 9aabe08512373..971f13d621efa 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py @@ -11,6 +11,7 @@ from dagster_graphql.implementation.fetch_runs import get_run_ids, get_runs, get_runs_count from dagster_graphql.implementation.utils import UserFacingGraphQLError +from dagster_graphql.schema.backfill import pipeline_execution_error_types from dagster_graphql.schema.errors import ( GrapheneInvalidPipelineRunsFilterError, GraphenePythonError, @@ -73,17 +74,28 @@ class Meta: class GrapheneLaunchRunResult(graphene.Union): class Meta: - from dagster_graphql.schema.backfill import pipeline_execution_error_types - types = launch_pipeline_run_result_types + pipeline_execution_error_types name = "LaunchRunResult" -class GrapheneLaunchRunReexecutionResult(graphene.Union): +class GrapheneLaunchMultipleRunsResult(graphene.ObjectType): + """Contains results from multiple pipeline launches.""" + + launchMultipleRunsResult = non_null_list(GrapheneLaunchRunResult) + + class Meta: + name = "LaunchMultipleRunsResult" + + +class GrapheneLaunchMultipleRunsResultOrError(graphene.Union): class Meta: - from dagster_graphql.schema.backfill import pipeline_execution_error_types + types = (GrapheneLaunchMultipleRunsResult, GraphenePythonError) + name = "LaunchMultipleRunsResultOrError" + +class GrapheneLaunchRunReexecutionResult(graphene.Union): + class Meta: types = launch_pipeline_run_result_types + pipeline_execution_error_types name = "LaunchRunReexecutionResult" @@ -213,6 +225,7 @@ def parse_run_config_input( types = [ GrapheneLaunchRunResult, + GrapheneLaunchMultipleRunsResult, GrapheneLaunchRunReexecutionResult, GrapheneLaunchPipelineRunSuccess, GrapheneLaunchRunSuccess, diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr index a3d7ea283166c..260759614f90a 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr @@ -5097,23 +5097,50 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.61e1eeaf22dac69e3d93b94002a5eb303d2c2151": { + "Shape.5e7f7baf52b7ed6a625b2f172c255b54642d5cf3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "return_hello", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"return_foo\": {}, \"return_hello_world\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.ac8a4423400648ec2ae58a67c8a877953652bf1f" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.61e1eeaf22dac69e3d93b94002a5eb303d2c2151", + "key": "Shape.5e7f7baf52b7ed6a625b2f172c255b54642d5cf3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -5143,50 +5170,32 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.cf1bfb906829a69c3b012b4858c1ebfe0080e350": { + "Shape.ac8a4423400648ec2ae58a67c8a877953652bf1f": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"return_hello\": {}}", - "description": "Configure runtime parameters for ops or assets.", + "description": null, "is_required": false, - "name": "ops", - "type_key": "Shape.61e1eeaf22dac69e3d93b94002a5eb303d2c2151" + "name": "return_foo", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "return_hello_world", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.cf1bfb906829a69c3b012b4858c1ebfe0080e350", + "key": "Shape.ac8a4423400648ec2ae58a67c8a877953652bf1f", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -5342,14 +5351,35 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "return_hello", - "solid_name": "return_hello", + "solid_def_name": "return_foo", + "solid_name": "return_foo", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "_foo", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "return_foo" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "return_hello_world", + "solid_name": "return_hello_world", "tags": {} } ] }, "description": null, - "graph_def_name": "no_config_job", + "graph_def_name": "no_config_chain_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -5388,10 +5418,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.cf1bfb906829a69c3b012b4858c1ebfe0080e350" + "root_config_key": "Shape.5e7f7baf52b7ed6a625b2f172c255b54642d5cf3" } ], - "name": "no_config_job", + "name": "no_config_chain_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -5409,7 +5439,41 @@ }, "description": null, "input_def_snaps": [], - "name": "return_hello", + "name": "return_foo", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "_foo" + } + ], + "name": "return_hello_world", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -5430,7 +5494,7 @@ ''' # --- # name: test_all_snapshot_ids[101] - '98a8e544c66ff337c2aef1334603df0a3b1c7434' + 'ab8f4b864ee53d2d9304b85f7a368aad7f678f29' # --- # name: test_all_snapshot_ids[102] ''' @@ -5593,29 +5657,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Selector.2571019f1a5201853d11032145ac3e534067f214": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "env", - "type_key": "String" - } - ], - "given_name": null, - "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", - "kind": { - "__enum__": "ConfigTypeKind.SELECTOR" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -5899,23 +5940,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "base_dir", - "type_key": "StringSourceType" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -5954,96 +5995,46 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + "is_required": true, + "name": "path", + "type_key": "String" } ], "given_name": null, - "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.47a1e206d1d386ec20b9f28378954bc9abedccef": { + "Shape.61e1eeaf22dac69e3d93b94002a5eb303d2c2151": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"no_multipartitions_1\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.b02f9c378d5f0f3ea347615e41637ef3f40b02dc" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" - } - ], - "given_name": null, - "key": "Shape.47a1e206d1d386ec20b9f28378954bc9abedccef", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "path", - "type_key": "String" + "is_required": false, + "name": "return_hello", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.61e1eeaf22dac69e3d93b94002a5eb303d2c2151", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -6073,110 +6064,50 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "Shape.cf1bfb906829a69c3b012b4858c1ebfe0080e350": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "dummy_io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"return_hello\": {}}", + "description": "Configure runtime parameters for ops or assets.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" - } - ], - "given_name": null, - "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "file", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.b02f9c378d5f0f3ea347615e41637ef3f40b02dc": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ + "name": "ops", + "type_key": "Shape.61e1eeaf22dac69e3d93b94002a5eb303d2c2151" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", "is_required": false, - "name": "no_multipartitions_1", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.b02f9c378d5f0f3ea347615e41637ef3f40b02dc", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", + "key": "Shape.cf1bfb906829a69c3b012b4858c1ebfe0080e350", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -6233,22 +6164,6 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null - }, - "StringSourceType": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": null, - "given_name": null, - "key": "StringSourceType", - "kind": { - "__enum__": "ConfigTypeKind.SCALAR_UNION" - }, - "scalar_kind": null, - "type_param_keys": [ - "String", - "Selector.2571019f1a5201853d11032145ac3e534067f214" - ] } } }, @@ -6348,14 +6263,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "no_multipartitions_1", - "solid_name": "no_multipartitions_1", + "solid_def_name": "return_hello", + "solid_name": "return_hello", "tags": {} } ] }, "description": null, - "graph_def_name": "no_multipartitions_job", + "graph_def_name": "no_config_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -6390,43 +6305,14 @@ "name": "config", "type_key": "Any" }, - "description": null, - "name": "dummy_io_manager" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" - }, - "description": null, - "name": "hanging_asset_resource" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.47a1e206d1d386ec20b9f28378954bc9abedccef" + "root_config_key": "Shape.cf1bfb906829a69c3b012b4858c1ebfe0080e350" } ], - "name": "no_multipartitions_job", - "run_tags": {}, + "name": "no_config_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -6444,7 +6330,7 @@ }, "description": null, "input_def_snaps": [], - "name": "no_multipartitions_1", + "name": "return_hello", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -6465,7 +6351,7 @@ ''' # --- # name: test_all_snapshot_ids[103] - 'f53f6915917e179ab89bae44c345e79603f8d42d' + '98a8e544c66ff337c2aef1334603df0a3b1c7434' # --- # name: test_all_snapshot_ids[104] ''' @@ -6628,6 +6514,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Selector.2571019f1a5201853d11032145ac3e534067f214": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "env", + "type_key": "String" + } + ], + "given_name": null, + "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -6911,7 +6820,62 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "base_dir", + "type_key": "StringSourceType" + } + ], + "given_name": null, + "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", + "is_required": false, + "name": "marker_to_close", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", + "is_required": false, + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + } + ], + "given_name": null, + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -6920,21 +6884,21 @@ "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "description": null, "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a": { + "Shape.47a1e206d1d386ec20b9f28378954bc9abedccef": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -6960,54 +6924,77 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"noop_op\": {}}", + "default_value_as_json_str": "{\"no_multipartitions_1\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.242592fa9f0be8d5908506e918e119be06358618" + "type_key": "Shape.b02f9c378d5f0f3ea347615e41637ef3f40b02dc" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", + "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", "description": "Configure how shared resources are implemented within a run.", "is_required": false, "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" } ], "given_name": null, - "key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a", + "key": "Shape.47a1e206d1d386ec20b9f28378954bc9abedccef", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.242592fa9f0be8d5908506e918e119be06358618": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "path", + "type_key": "String" + } + ], + "given_name": null, + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "noop_op", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.242592fa9f0be8d5908506e918e119be06358618", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -7016,30 +7003,39 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "[DEPRECATED]", + "description": null, "is_required": false, - "name": "marker_to_close", - "type_key": "String" + "name": "dummy_io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "{\"config\": {}}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + "name": "io_manager", + "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -7050,19 +7046,42 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "path", + "name": "file", "type_key": "String" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.b02f9c378d5f0f3ea347615e41637ef3f40b02dc": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "no_multipartitions_1", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.b02f9c378d5f0f3ea347615e41637ef3f40b02dc", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -7072,13 +7091,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -7135,6 +7154,22 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null + }, + "StringSourceType": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "StringSourceType", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "String", + "Selector.2571019f1a5201853d11032145ac3e534067f214" + ] } } }, @@ -7234,14 +7269,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "noop_op", - "solid_name": "noop_op", + "solid_def_name": "no_multipartitions_1", + "solid_name": "no_multipartitions_1", "tags": {} } ] }, "description": null, - "graph_def_name": "noop_job", + "graph_def_name": "no_multipartitions_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -7276,14 +7311,43 @@ "name": "config", "type_key": "Any" }, + "description": null, + "name": "dummy_io_manager" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + }, + "description": null, + "name": "hanging_asset_resource" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a" + "root_config_key": "Shape.47a1e206d1d386ec20b9f28378954bc9abedccef" } ], - "name": "noop_job", + "name": "no_multipartitions_job", + "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -7301,7 +7365,7 @@ }, "description": null, "input_def_snaps": [], - "name": "noop_op", + "name": "no_multipartitions_1", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -7322,7 +7386,7 @@ ''' # --- # name: test_all_snapshot_ids[105] - 'd65a072229c6e8fc80db02c8d06067f3b0b48305' + 'f53f6915917e179ab89bae44c345e79603f8d42d' # --- # name: test_all_snapshot_ids[106] ''' @@ -7485,29 +7549,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Selector.2571019f1a5201853d11032145ac3e534067f214": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "env", - "type_key": "String" - } - ], - "given_name": null, - "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", - "kind": { - "__enum__": "ConfigTypeKind.SELECTOR" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -7791,108 +7832,80 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "base_dir", - "type_key": "StringSourceType" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "[DEPRECATED]", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "marker_to_close", - "type_key": "String" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" - } - ], - "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"noop_op\": {}}", + "description": "Configure runtime parameters for ops or assets.", "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - } - ], - "given_name": null, - "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ + "name": "ops", + "type_key": "Shape.242592fa9f0be8d5908506e918e119be06358618" + }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "path", - "type_key": "String" + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.72aa8546e0029d52a6bfce93acd5d0b917139098": { + "Shape.242592fa9f0be8d5908506e918e119be06358618": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -7903,42 +7916,19 @@ "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "asset_yields_observation", + "name": "noop_op", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.72aa8546e0029d52a6bfce93acd5d0b917139098", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - } - ], - "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.242592fa9f0be8d5908506e918e119be06358618", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -7947,39 +7937,30 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "dummy_io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "description": "[DEPRECATED]", "is_required": false, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + "name": "marker_to_close", + "type_key": "String" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" } ], "given_name": null, - "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -7990,19 +7971,19 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "file", + "name": "path", "type_key": "String" } ], "given_name": null, - "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -8012,13 +7993,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, + "is_required": false, "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + "type_key": "Any" } ], "given_name": null, - "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -8038,56 +8019,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.db26c3a200f5a9ca97fcd23a0ff51221959871b2": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"asset_yields_observation\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.72aa8546e0029d52a6bfce93acd5d0b917139098" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" - } - ], - "given_name": null, - "key": "Shape.db26c3a200f5a9ca97fcd23a0ff51221959871b2", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { "__class__": "ConfigTypeSnap", "description": null, @@ -8125,22 +8056,6 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null - }, - "StringSourceType": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": null, - "given_name": null, - "key": "StringSourceType", - "kind": { - "__enum__": "ConfigTypeKind.SCALAR_UNION" - }, - "scalar_kind": null, - "type_param_keys": [ - "String", - "Selector.2571019f1a5201853d11032145ac3e534067f214" - ] } } }, @@ -8240,14 +8155,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "asset_yields_observation", - "solid_name": "asset_yields_observation", + "solid_def_name": "noop_op", + "solid_name": "noop_op", "tags": {} } ] }, "description": null, - "graph_def_name": "observation_job", + "graph_def_name": "noop_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -8282,43 +8197,14 @@ "name": "config", "type_key": "Any" }, - "description": null, - "name": "dummy_io_manager" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" - }, - "description": null, - "name": "hanging_asset_resource" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.db26c3a200f5a9ca97fcd23a0ff51221959871b2" + "root_config_key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a" } ], - "name": "observation_job", - "run_tags": {}, + "name": "noop_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -8336,7 +8222,7 @@ }, "description": null, "input_def_snaps": [], - "name": "asset_yields_observation", + "name": "noop_op", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -8357,7 +8243,7 @@ ''' # --- # name: test_all_snapshot_ids[107] - '77bb631e77718a35cfd31049d03241709f9454ea' + 'd65a072229c6e8fc80db02c8d06067f3b0b48305' # --- # name: test_all_snapshot_ids[108] ''' @@ -8520,6 +8406,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Selector.2571019f1a5201853d11032145ac3e534067f214": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "env", + "type_key": "String" + } + ], + "given_name": null, + "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -8803,41 +8712,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.22ef5f214708cb30cd4940652ff6f8f900c765d6": { + "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "dummy_io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "base_dir", + "type_key": "StringSourceType" } ], "given_name": null, - "key": "Shape.22ef5f214708cb30cd4940652ff6f8f900c765d6", + "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -8876,6 +8767,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + } + ], + "given_name": null, + "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, @@ -8899,6 +8813,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.72aa8546e0029d52a6bfce93acd5d0b917139098": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "asset_yields_observation", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.72aa8546e0029d52a6bfce93acd5d0b917139098", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, @@ -8922,6 +8859,47 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "dummy_io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {}}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + } + ], + "given_name": null, + "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { "__class__": "ConfigTypeSnap", "description": null, @@ -8968,7 +8946,20 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.cb9d09e190ef60d4174654b8fc26e7574f87d657": { + "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [], + "given_name": null, + "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.db26c3a200f5a9ca97fcd23a0ff51221959871b2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -8994,60 +8985,24 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"output_then_hang_asset\": {}}", + "default_value_as_json_str": "{\"asset_yields_observation\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.e04fad9a05c90983540f9c3eb316ede838d946e2" + "type_key": "Shape.72aa8546e0029d52a6bfce93acd5d0b917139098" }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Configure how shared resources are implemented within a run.", - "is_required": true, - "name": "resources", - "type_key": "Shape.22ef5f214708cb30cd4940652ff6f8f900c765d6" - } - ], - "given_name": null, - "key": "Shape.cb9d09e190ef60d4174654b8fc26e7574f87d657", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [], - "given_name": null, - "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.e04fad9a05c90983540f9c3eb316ede838d946e2": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", + "description": "Configure how shared resources are implemented within a run.", "is_required": false, - "name": "output_then_hang_asset", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "resources", + "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" } ], "given_name": null, - "key": "Shape.e04fad9a05c90983540f9c3eb316ede838d946e2", + "key": "Shape.db26c3a200f5a9ca97fcd23a0ff51221959871b2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -9091,6 +9046,22 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null + }, + "StringSourceType": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "StringSourceType", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "String", + "Selector.2571019f1a5201853d11032145ac3e534067f214" + ] } } }, @@ -9190,14 +9161,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "output_then_hang_asset", - "solid_name": "output_then_hang_asset", + "solid_def_name": "asset_yields_observation", + "solid_name": "asset_yields_observation", "tags": {} } ] }, "description": null, - "graph_def_name": "output_then_hang_job", + "graph_def_name": "observation_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -9253,21 +9224,21 @@ "__class__": "ResourceDefSnap", "config_field_snap": { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, "is_required": false, "name": "config", - "type_key": "Any" + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.cb9d09e190ef60d4174654b8fc26e7574f87d657" + "root_config_key": "Shape.db26c3a200f5a9ca97fcd23a0ff51221959871b2" } ], - "name": "output_then_hang_job", + "name": "observation_job", "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", @@ -9286,7 +9257,7 @@ }, "description": null, "input_def_snaps": [], - "name": "output_then_hang_asset", + "name": "asset_yields_observation", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -9297,9 +9268,7 @@ "name": "result" } ], - "required_resource_keys": [ - "hanging_asset_resource" - ], + "required_resource_keys": [], "tags": {} } ] @@ -9309,7 +9278,7 @@ ''' # --- # name: test_all_snapshot_ids[109] - 'c5cbcfa0e6f1c8658773ba21a9a70fd1b7c517eb' + '77bb631e77718a35cfd31049d03241709f9454ea' # --- # name: test_all_snapshot_ids[10] ''' @@ -10266,29 +10235,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Selector.2571019f1a5201853d11032145ac3e534067f214": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "env", - "type_key": "String" - } - ], - "given_name": null, - "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", - "kind": { - "__enum__": "ConfigTypeKind.SELECTOR" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -10572,23 +10518,41 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { + "Shape.22ef5f214708cb30cd4940652ff6f8f900c765d6": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "dummy_io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, "description": null, + "is_required": true, + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "base_dir", - "type_key": "StringSourceType" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", + "key": "Shape.22ef5f214708cb30cd4940652ff6f8f900c765d6", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -10627,53 +10591,53 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + "is_required": true, + "name": "path", + "type_key": "String" } ], "given_name": null, - "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.464daa5cd8a2371e65188e415cb75856137daddb": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "yield_partition_materialization", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.464daa5cd8a2371e65188e415cb75856137daddb", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -10684,19 +10648,19 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "path", + "name": "file", "type_key": "String" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -10706,100 +10670,63 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "Shape.cb9d09e190ef60d4174654b8fc26e7574f87d657": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "dummy_io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"output_then_hang_asset\": {}}", + "description": "Configure runtime parameters for ops or assets.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" - } - ], - "given_name": null, - "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "file", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ + "name": "ops", + "type_key": "Shape.e04fad9a05c90983540f9c3eb316ede838d946e2" + }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, + "description": "Configure how shared resources are implemented within a run.", "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + "name": "resources", + "type_key": "Shape.22ef5f214708cb30cd4940652ff6f8f900c765d6" } ], "given_name": null, - "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", + "key": "Shape.cb9d09e190ef60d4174654b8fc26e7574f87d657", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -10819,73 +10746,46 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { + "Shape.e04fad9a05c90983540f9c3eb316ede838d946e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "console", - "type_key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a" + "name": "output_then_hang_asset", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b", + "key": "Shape.e04fad9a05c90983540f9c3eb316ede838d946e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.f851693f4be830bdd04a202ef75408cde605494f": { + "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"yield_partition_materialization\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.464daa5cd8a2371e65188e415cb75856137daddb" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", - "description": "Configure how shared resources are implemented within a run.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "resources", - "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" + "name": "console", + "type_key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a" } ], "given_name": null, - "key": "Shape.f851693f4be830bdd04a202ef75408cde605494f", + "key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -10906,22 +10806,6 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null - }, - "StringSourceType": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": null, - "given_name": null, - "key": "StringSourceType", - "kind": { - "__enum__": "ConfigTypeKind.SCALAR_UNION" - }, - "scalar_kind": null, - "type_param_keys": [ - "String", - "Selector.2571019f1a5201853d11032145ac3e534067f214" - ] } } }, @@ -11021,14 +10905,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "yield_partition_materialization", - "solid_name": "yield_partition_materialization", + "solid_def_name": "output_then_hang_asset", + "solid_name": "output_then_hang_asset", "tags": {} } ] }, "description": null, - "graph_def_name": "partition_materialization_job", + "graph_def_name": "output_then_hang_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -11084,21 +10968,21 @@ "__class__": "ResourceDefSnap", "config_field_snap": { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + "type_key": "Any" }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.f851693f4be830bdd04a202ef75408cde605494f" + "root_config_key": "Shape.cb9d09e190ef60d4174654b8fc26e7574f87d657" } ], - "name": "partition_materialization_job", + "name": "output_then_hang_job", "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", @@ -11117,7 +11001,7 @@ }, "description": null, "input_def_snaps": [], - "name": "yield_partition_materialization", + "name": "output_then_hang_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -11128,7 +11012,9 @@ "name": "result" } ], - "required_resource_keys": [], + "required_resource_keys": [ + "hanging_asset_resource" + ], "tags": {} } ] @@ -11138,7 +11024,7 @@ ''' # --- # name: test_all_snapshot_ids[111] - '8b0a6d0b6366a3178821e74b13782124ffa5d0fd' + 'c5cbcfa0e6f1c8658773ba21a9a70fd1b7c517eb' # --- # name: test_all_snapshot_ids[112] ''' @@ -11301,6 +11187,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Selector.2571019f1a5201853d11032145ac3e534067f214": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "env", + "type_key": "String" + } + ], + "given_name": null, + "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -11584,78 +11493,101 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "base_dir", + "type_key": "StringSourceType" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.2a96bc04c9f379d60bccfdbc745bf93c8d1f867d": { + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", + "is_required": false, + "name": "marker_to_close", + "type_key": "String" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", "is_required": false, - "name": "op_partitioned_asset", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" } ], "given_name": null, - "key": "Shape.2a96bc04c9f379d60bccfdbc745bf93c8d1f867d", + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "[DEPRECATED]", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "marker_to_close", - "type_key": "String" - }, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + } + ], + "given_name": null, + "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.464daa5cd8a2371e65188e415cb75856137daddb": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + "name": "yield_partition_materialization", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.464daa5cd8a2371e65188e415cb75856137daddb", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -11708,50 +11640,87 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.78557f5eae761798e03399f52f044879af8415a3": { + "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + "name": "dummy_io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"op_partitioned_asset\": {}}", - "description": "Configure runtime parameters for ops or assets.", + "default_value_as_json_str": "{\"config\": {}}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "ops", - "type_key": "Shape.2a96bc04c9f379d60bccfdbc745bf93c8d1f867d" - }, + "name": "io_manager", + "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + } + ], + "given_name": null, + "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "file", + "type_key": "String" } ], "given_name": null, - "key": "Shape.78557f5eae761798e03399f52f044879af8415a3", + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + } + ], + "given_name": null, + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -11794,6 +11763,56 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.f851693f4be830bdd04a202ef75408cde605494f": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"yield_partition_materialization\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.464daa5cd8a2371e65188e415cb75856137daddb" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" + } + ], + "given_name": null, + "key": "Shape.f851693f4be830bdd04a202ef75408cde605494f", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -11808,6 +11827,22 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null + }, + "StringSourceType": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "StringSourceType", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "String", + "Selector.2571019f1a5201853d11032145ac3e534067f214" + ] } } }, @@ -11907,14 +11942,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_partitioned_asset", - "solid_name": "op_partitioned_asset", + "solid_def_name": "yield_partition_materialization", + "solid_name": "yield_partition_materialization", "tags": {} } ] }, "description": null, - "graph_def_name": "partitioned_asset_job", + "graph_def_name": "partition_materialization_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -11949,14 +11984,43 @@ "name": "config", "type_key": "Any" }, + "description": null, + "name": "dummy_io_manager" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + }, + "description": null, + "name": "hanging_asset_resource" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.78557f5eae761798e03399f52f044879af8415a3" + "root_config_key": "Shape.f851693f4be830bdd04a202ef75408cde605494f" } ], - "name": "partitioned_asset_job", + "name": "partition_materialization_job", + "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -11974,7 +12038,7 @@ }, "description": null, "input_def_snaps": [], - "name": "op_partitioned_asset", + "name": "yield_partition_materialization", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -11995,7 +12059,7 @@ ''' # --- # name: test_all_snapshot_ids[113] - '88c326b7e07c8c537ecd8086fd6429436a46c66f' + '8b0a6d0b6366a3178821e74b13782124ffa5d0fd' # --- # name: test_all_snapshot_ids[114] ''' @@ -12464,23 +12528,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.4384fce472621a1d43c54ff7e52b02891791103f": { + "Shape.2a96bc04c9f379d60bccfdbc745bf93c8d1f867d": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "foo", - "type_key": "String" + "is_required": false, + "name": "op_partitioned_asset", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.4384fce472621a1d43c54ff7e52b02891791103f", + "key": "Shape.2a96bc04c9f379d60bccfdbc745bf93c8d1f867d", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -12565,23 +12629,50 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.77b7c2aa740a9b03628c25cc1441dfe267c703f8": { + "Shape.78557f5eae761798e03399f52f044879af8415a3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.4384fce472621a1d43c54ff7e52b02891791103f" + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"op_partitioned_asset\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.2a96bc04c9f379d60bccfdbc745bf93c8d1f867d" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.77b7c2aa740a9b03628c25cc1441dfe267c703f8", + "key": "Shape.78557f5eae761798e03399f52f044879af8415a3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -12624,79 +12715,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.e8a5a728f50d8ebcb6d59955a173dd127aaad2d4": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "the_op", - "type_key": "Shape.77b7c2aa740a9b03628c25cc1441dfe267c703f8" - } - ], - "given_name": null, - "key": "Shape.e8a5a728f50d8ebcb6d59955a173dd127aaad2d4", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.ec386eda01844b8b94d4c1c10159dc060b7e480e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Configure runtime parameters for ops or assets.", - "is_required": true, - "name": "ops", - "type_key": "Shape.e8a5a728f50d8ebcb6d59955a173dd127aaad2d4" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" - } - ], - "given_name": null, - "key": "Shape.ec386eda01844b8b94d4c1c10159dc060b7e480e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -12810,14 +12828,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "the_op", - "solid_name": "the_op", + "solid_def_name": "op_partitioned_asset", + "solid_name": "op_partitioned_asset", "tags": {} } ] }, "description": null, - "graph_def_name": "req_config_job", + "graph_def_name": "partitioned_asset_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -12856,10 +12874,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.ec386eda01844b8b94d4c1c10159dc060b7e480e" + "root_config_key": "Shape.78557f5eae761798e03399f52f044879af8415a3" } ], - "name": "req_config_job", + "name": "partitioned_asset_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -12871,13 +12889,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, + "is_required": false, "name": "config", - "type_key": "Shape.4384fce472621a1d43c54ff7e52b02891791103f" + "type_key": "Any" }, "description": null, "input_def_snaps": [], - "name": "the_op", + "name": "op_partitioned_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -12898,7 +12916,7 @@ ''' # --- # name: test_all_snapshot_ids[115] - 'a73f4941b2735f3a261d9617abaaed09a88aaebc' + '88c326b7e07c8c537ecd8086fd6429436a46c66f' # --- # name: test_all_snapshot_ids[116] ''' @@ -13344,20 +13362,11 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.13e52aa6878a8e9f6b5fea1f6086cb52486e3c95": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "R1", - "type_key": "Shape.fe2c8a3955b895767072f0aa1d243b6e1714df90" - }, { "__class__": "ConfigFieldSnap", "default_provided": true, @@ -13369,57 +13378,30 @@ } ], "given_name": null, - "key": "Shape.13e52aa6878a8e9f6b5fea1f6086cb52486e3c95", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.1f049499585829308eb97bdfcb96ec7cad460ad6": { + "Shape.4384fce472621a1d43c54ff7e52b02891791103f": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"op_with_required_resource\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.e184cc449eaa96b4c59f30cbdf34d7efa47bb273" - }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "Configure how shared resources are implemented within a run.", + "description": null, "is_required": true, - "name": "resources", - "type_key": "Shape.13e52aa6878a8e9f6b5fea1f6086cb52486e3c95" + "name": "foo", + "type_key": "String" } ], "given_name": null, - "key": "Shape.1f049499585829308eb97bdfcb96ec7cad460ad6", + "key": "Shape.4384fce472621a1d43c54ff7e52b02891791103f", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -13504,36 +13486,36 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { + "Shape.77b7c2aa740a9b03628c25cc1441dfe267c703f8": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, - "fields": [], + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.4384fce472621a1d43c54ff7e52b02891791103f" + } + ], "given_name": null, - "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", + "key": "Shape.77b7c2aa740a9b03628c25cc1441dfe267c703f8", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.e184cc449eaa96b4c59f30cbdf34d7efa47bb273": { + "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "op_with_required_resource", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], + "fields": [], "given_name": null, - "key": "Shape.e184cc449eaa96b4c59f30cbdf34d7efa47bb273", + "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -13563,7 +13545,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.fe2c8a3955b895767072f0aa1d243b6e1714df90": { + "Shape.e8a5a728f50d8ebcb6d59955a173dd127aaad2d4": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -13574,12 +13556,62 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "config", - "type_key": "Int" + "name": "the_op", + "type_key": "Shape.77b7c2aa740a9b03628c25cc1441dfe267c703f8" } ], "given_name": null, - "key": "Shape.fe2c8a3955b895767072f0aa1d243b6e1714df90", + "key": "Shape.e8a5a728f50d8ebcb6d59955a173dd127aaad2d4", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.ec386eda01844b8b94d4c1c10159dc060b7e480e": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Configure runtime parameters for ops or assets.", + "is_required": true, + "name": "ops", + "type_key": "Shape.e8a5a728f50d8ebcb6d59955a173dd127aaad2d4" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + } + ], + "given_name": null, + "key": "Shape.ec386eda01844b8b94d4c1c10159dc060b7e480e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -13699,14 +13731,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_with_required_resource", - "solid_name": "op_with_required_resource", + "solid_def_name": "the_op", + "solid_name": "the_op", "tags": {} } ] }, "description": null, - "graph_def_name": "required_resource_config_job", + "graph_def_name": "req_config_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -13730,20 +13762,6 @@ ], "name": "default", "resource_def_snaps": [ - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Int" - }, - "description": null, - "name": "R1" - }, { "__class__": "ResourceDefSnap", "config_field_snap": { @@ -13759,10 +13777,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.1f049499585829308eb97bdfcb96ec7cad460ad6" + "root_config_key": "Shape.ec386eda01844b8b94d4c1c10159dc060b7e480e" } ], - "name": "required_resource_config_job", + "name": "req_config_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -13774,13 +13792,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Shape.4384fce472621a1d43c54ff7e52b02891791103f" }, "description": null, "input_def_snaps": [], - "name": "op_with_required_resource", + "name": "the_op", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -13791,9 +13809,7 @@ "name": "result" } ], - "required_resource_keys": [ - "R1" - ], + "required_resource_keys": [], "tags": {} } ] @@ -13803,7 +13819,7 @@ ''' # --- # name: test_all_snapshot_ids[117] - '11f4785f61153adc6cca6935748af3807b59eee9' + 'a73f4941b2735f3a261d9617abaaed09a88aaebc' # --- # name: test_all_snapshot_ids[118] ''' @@ -14226,7 +14242,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.0f87ccd9832d8df6e0ba99cc8ba1302be3dc7b63": { + "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -14234,54 +14250,54 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "R1", - "type_key": "Shape.9d71d8e2083b6468a26a8556e56288deec792b4e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", + "description": "The default colored console logger.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" } ], "given_name": null, - "key": "Shape.0f87ccd9832d8df6e0ba99cc8ba1302be3dc7b63", + "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { + "Shape.13e52aa6878a8e9f6b5fea1f6086cb52486e3c95": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "R1", + "type_key": "Shape.fe2c8a3955b895767072f0aa1d243b6e1714df90" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", - "description": "The default colored console logger.", + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "config", - "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", + "key": "Shape.13e52aa6878a8e9f6b5fea1f6086cb52486e3c95", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.3ccfab37537e163ad2b3b0b2bb47397515e13ac6": { + "Shape.1f049499585829308eb97bdfcb96ec7cad460ad6": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -14315,16 +14331,16 @@ }, { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"R1\": {}, \"io_manager\": {}}", + "default_provided": false, + "default_value_as_json_str": null, "description": "Configure how shared resources are implemented within a run.", - "is_required": false, + "is_required": true, "name": "resources", - "type_key": "Shape.0f87ccd9832d8df6e0ba99cc8ba1302be3dc7b63" + "type_key": "Shape.13e52aa6878a8e9f6b5fea1f6086cb52486e3c95" } ], "given_name": null, - "key": "Shape.3ccfab37537e163ad2b3b0b2bb47397515e13ac6", + "key": "Shape.1f049499585829308eb97bdfcb96ec7cad460ad6", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -14409,29 +14425,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.9d71d8e2083b6468a26a8556e56288deec792b4e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Int" - } - ], - "given_name": null, - "key": "Shape.9d71d8e2083b6468a26a8556e56288deec792b4e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { "__class__": "ConfigTypeSnap", "description": null, @@ -14491,6 +14484,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.fe2c8a3955b895767072f0aa1d243b6e1714df90": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Int" + } + ], + "given_name": null, + "key": "Shape.fe2c8a3955b895767072f0aa1d243b6e1714df90", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -14611,7 +14627,7 @@ ] }, "description": null, - "graph_def_name": "required_resource_job", + "graph_def_name": "required_resource_config_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -14642,7 +14658,7 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", "type_key": "Int" }, @@ -14664,10 +14680,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.3ccfab37537e163ad2b3b0b2bb47397515e13ac6" + "root_config_key": "Shape.1f049499585829308eb97bdfcb96ec7cad460ad6" } ], - "name": "required_resource_job", + "name": "required_resource_config_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -14708,7 +14724,7 @@ ''' # --- # name: test_all_snapshot_ids[119] - '2b43566d7b3de2138b12c91b6953d6f1ddde0140' + '11f4785f61153adc6cca6935748af3807b59eee9' # --- # name: test_all_snapshot_ids[11] 'cd81ec29a7fde8a337dc04fb109dd707a2962d18' @@ -15134,6 +15150,38 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.0f87ccd9832d8df6e0ba99cc8ba1302be3dc7b63": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "R1", + "type_key": "Shape.9d71d8e2083b6468a26a8556e56288deec792b4e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.0f87ccd9832d8df6e0ba99cc8ba1302be3dc7b63", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { "__class__": "ConfigTypeSnap", "description": null, @@ -15157,73 +15205,50 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.24ee1fbfd9ec8adc8f4e5ab30e651b52389c861e": { + "Shape.3ccfab37537e163ad2b3b0b2bb47397515e13ac6": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "get_input_one", - "type_key": "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "get_input_two", - "type_key": "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c" + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "return_one", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"op_with_required_resource\": {}}", + "description": "Configure runtime parameters for ops or assets.", "is_required": false, - "name": "sum_inputs", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.24ee1fbfd9ec8adc8f4e5ab30e651b52389c861e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ + "name": "ops", + "type_key": "Shape.e184cc449eaa96b4c59f30cbdf34d7efa47bb273" + }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64" + "default_provided": true, + "default_value_as_json_str": "{\"R1\": {}, \"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.0f87ccd9832d8df6e0ba99cc8ba1302be3dc7b63" } ], "given_name": null, - "key": "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c", + "key": "Shape.3ccfab37537e163ad2b3b0b2bb47397515e13ac6", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -15285,56 +15310,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.64df299302dcce81c8530dd4f523dc0c9c3eaf61": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Configure runtime parameters for ops or assets.", - "is_required": true, - "name": "ops", - "type_key": "Shape.24ee1fbfd9ec8adc8f4e5ab30e651b52389c861e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"disable_gc\": {}, \"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.c4c8567ee7c3e125a72762d0a1a7d714c1b83668" - } - ], - "given_name": null, - "key": "Shape.64df299302dcce81c8530dd4f523dc0c9c3eaf61", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, @@ -15358,32 +15333,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.c4c8567ee7c3e125a72762d0a1a7d714c1b83668": { + "Shape.9d71d8e2083b6468a26a8556e56288deec792b4e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "disable_gc", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Int" } ], "given_name": null, - "key": "Shape.c4c8567ee7c3e125a72762d0a1a7d714c1b83668", + "key": "Shape.9d71d8e2083b6468a26a8556e56288deec792b4e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -15403,23 +15369,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64": { + "Shape.e184cc449eaa96b4c59f30cbdf34d7efa47bb273": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "wait_to_terminate", - "type_key": "Bool" + "is_required": false, + "name": "op_with_required_resource", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64", + "key": "Shape.e184cc449eaa96b4c59f30cbdf34d7efa47bb273", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -15558,93 +15524,18 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "one", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "return_one" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "get_input_one", - "solid_name": "get_input_one", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "one", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "return_one" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "get_input_two", - "solid_name": "get_input_two", - "tags": {} - }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "return_one", - "solid_name": "return_one", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "input_one", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "get_input_one" - } - ] - }, - { - "__class__": "InputDependencySnap", - "input_name": "input_two", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "get_input_two" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "sum_inputs", - "solid_name": "sum_inputs", + "solid_def_name": "op_with_required_resource", + "solid_name": "op_with_required_resource", "tags": {} } ] }, "description": null, - "graph_def_name": "retry_multi_input_early_terminate_job", + "graph_def_name": "required_resource_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -15677,10 +15568,10 @@ "description": null, "is_required": false, "name": "config", - "type_key": "Any" + "type_key": "Int" }, "description": null, - "name": "disable_gc" + "name": "R1" }, { "__class__": "ResourceDefSnap", @@ -15697,86 +15588,14 @@ "name": "io_manager" } ], - "root_config_key": "Shape.64df299302dcce81c8530dd4f523dc0c9c3eaf61" + "root_config_key": "Shape.3ccfab37537e163ad2b3b0b2bb47397515e13ac6" } ], - "name": "retry_multi_input_early_terminate_job", + "name": "required_resource_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "one" - } - ], - "name": "get_input_one", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Int", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [ - "disable_gc" - ], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "one" - } - ], - "name": "get_input_two", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Int", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [ - "disable_gc" - ], - "tags": {} - }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -15790,58 +15609,20 @@ }, "description": null, "input_def_snaps": [], - "name": "return_one", + "name": "op_with_required_resource", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Int", + "dagster_type_key": "Any", "description": null, "is_dynamic": false, "is_required": true, "name": "result" } ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "input_one" - }, - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "input_two" - } - ], - "name": "sum_inputs", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Int", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } + "required_resource_keys": [ + "R1" ], - "required_resource_keys": [], "tags": {} } ] @@ -15851,7 +15632,7 @@ ''' # --- # name: test_all_snapshot_ids[121] - 'b07b89c1664d4a248bc1a39974a91b31d8956c54' + '2b43566d7b3de2138b12c91b6953d6f1ddde0140' # --- # name: test_all_snapshot_ids[122] ''' @@ -16274,29 +16055,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "fail", - "type_key": "Bool" - } - ], - "given_name": null, - "key": "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { "__class__": "ConfigTypeSnap", "description": null, @@ -16320,73 +16078,73 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.24ee1fbfd9ec8adc8f4e5ab30e651b52389c861e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "get_input_one", + "type_key": "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "get_input_two", + "type_key": "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "description": null, "is_required": false, - "name": "io_manager", + "name": "return_one", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "sum_inputs", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.24ee1fbfd9ec8adc8f4e5ab30e651b52389c861e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.381bcdb53d3182896a9a86cdfc9f625f62080106": { + "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "Configure runtime parameters for ops or assets.", + "description": null, "is_required": true, - "name": "ops", - "type_key": "Shape.857c24c238fce20e7bf57b59726c416b42738942" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "config", + "type_key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64" } ], "given_name": null, - "key": "Shape.381bcdb53d3182896a9a86cdfc9f625f62080106", + "key": "Shape.2e830fb24dc1eacc9bfa63a634b5715b46d81a2c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -16448,30 +16206,57 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.64df299302dcce81c8530dd4f523dc0c9c3eaf61": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, + "description": "Configure runtime parameters for ops or assets.", + "is_required": true, + "name": "ops", + "type_key": "Shape.24ee1fbfd9ec8adc8f4e5ab30e651b52389c861e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"disable_gc\": {}, \"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", "is_required": false, - "name": "config", - "type_key": "Any" + "name": "resources", + "type_key": "Shape.c4c8567ee7c3e125a72762d0a1a7d714c1b83668" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.64df299302dcce81c8530dd4f523dc0c9c3eaf61", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.857c24c238fce20e7bf57b59726c416b42738942": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -16481,94 +16266,81 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "can_fail", - "type_key": "Shape.a62c224a9a675b7b35544be496fa9f28c429c6b6" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "child_fail", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, "is_required": false, - "name": "child_multi_skip", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "child_skip", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, + "name": "config", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.c4c8567ee7c3e125a72762d0a1a7d714c1b83668": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "grandchild_fail", + "name": "disable_gc", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "multi", + "name": "io_manager", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.857c24c238fce20e7bf57b59726c416b42738942", + "key": "Shape.c4c8567ee7c3e125a72762d0a1a7d714c1b83668", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.a62c224a9a675b7b35544be496fa9f28c429c6b6": { + "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9" - } - ], + "fields": [], "given_name": null, - "key": "Shape.a62c224a9a675b7b35544be496fa9f28c429c6b6", + "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { + "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, - "fields": [], + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "wait_to_terminate", + "type_key": "Bool" + } + ], "given_name": null, - "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", + "key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -16712,20 +16484,20 @@ "input_dep_snaps": [ { "__class__": "InputDependencySnap", - "input_name": "inp", + "input_name": "one", "is_dynamic_collect": false, "upstream_output_snaps": [ { "__class__": "OutputHandleSnap", - "output_name": "success", - "solid_name": "multi" + "output_name": "result", + "solid_name": "return_one" } ] } ], "is_dynamic_mapped": false, - "solid_def_name": "can_fail", - "solid_name": "can_fail", + "solid_def_name": "get_input_one", + "solid_name": "get_input_one", "tags": {} }, { @@ -16733,41 +16505,28 @@ "input_dep_snaps": [ { "__class__": "InputDependencySnap", - "input_name": "value", + "input_name": "one", "is_dynamic_collect": false, "upstream_output_snaps": [ { "__class__": "OutputHandleSnap", - "output_name": "start_fail", - "solid_name": "can_fail" + "output_name": "result", + "solid_name": "return_one" } ] } ], "is_dynamic_mapped": false, - "solid_def_name": "passthrough", - "solid_name": "child_fail", + "solid_def_name": "get_input_two", + "solid_name": "get_input_two", "tags": {} }, { "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "start", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "skip", - "solid_name": "multi" - } - ] - } - ], + "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "no_output", - "solid_name": "child_multi_skip", + "solid_def_name": "return_one", + "solid_name": "return_one", "tags": {} }, { @@ -16775,55 +16534,38 @@ "input_dep_snaps": [ { "__class__": "InputDependencySnap", - "input_name": "start", + "input_name": "input_one", "is_dynamic_collect": false, "upstream_output_snaps": [ { "__class__": "OutputHandleSnap", - "output_name": "start_skip", - "solid_name": "can_fail" + "output_name": "result", + "solid_name": "get_input_one" } ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "no_output", - "solid_name": "child_skip", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ + }, { "__class__": "InputDependencySnap", - "input_name": "start", + "input_name": "input_two", "is_dynamic_collect": false, "upstream_output_snaps": [ { "__class__": "OutputHandleSnap", "output_name": "result", - "solid_name": "child_fail" + "solid_name": "get_input_two" } ] } ], "is_dynamic_mapped": false, - "solid_def_name": "no_output", - "solid_name": "grandchild_fail", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "multi", - "solid_name": "multi", + "solid_def_name": "sum_inputs", + "solid_name": "sum_inputs", "tags": {} } ] }, "description": null, - "graph_def_name": "retry_multi_output_job", + "graph_def_name": "retry_multi_input_early_terminate_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -16847,6 +16589,20 @@ ], "name": "default", "resource_def_snaps": [ + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "name": "disable_gc" + }, { "__class__": "ResourceDefSnap", "config_field_snap": { @@ -16862,10 +16618,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.381bcdb53d3182896a9a86cdfc9f625f62080106" + "root_config_key": "Shape.64df299302dcce81c8530dd4f523dc0c9c3eaf61" } ], - "name": "retry_multi_output_job", + "name": "retry_multi_input_early_terminate_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -16879,37 +16635,31 @@ "description": null, "is_required": true, "name": "config", - "type_key": "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9" + "type_key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64" }, "description": null, "input_def_snaps": [ { "__class__": "InputDefSnap", - "dagster_type_key": "String", + "dagster_type_key": "Int", "description": null, - "name": "inp" + "name": "one" } ], - "name": "can_fail", + "name": "get_input_one", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "String", - "description": null, - "is_dynamic": false, - "is_required": false, - "name": "start_fail" - }, - { - "__class__": "OutputDefSnap", - "dagster_type_key": "String", + "dagster_type_key": "Int", "description": null, "is_dynamic": false, - "is_required": false, - "name": "start_skip" + "is_required": true, + "name": "result" } ], - "required_resource_keys": [], + "required_resource_keys": [ + "disable_gc" + ], "tags": {} }, { @@ -16919,32 +16669,33 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Shape.dd25751df5066ca2b7e65e4907f386f04b08af64" }, "description": null, - "input_def_snaps": [], - "name": "multi", - "output_def_snaps": [ + "input_def_snaps": [ { - "__class__": "OutputDefSnap", - "dagster_type_key": "String", + "__class__": "InputDefSnap", + "dagster_type_key": "Int", "description": null, - "is_dynamic": false, - "is_required": false, - "name": "success" - }, + "name": "one" + } + ], + "name": "get_input_two", + "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "String", + "dagster_type_key": "Int", "description": null, "is_dynamic": false, - "is_required": false, - "name": "skip" + "is_required": true, + "name": "result" } ], - "required_resource_keys": [], + "required_resource_keys": [ + "disable_gc" + ], "tags": {} }, { @@ -16959,16 +16710,18 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ + "input_def_snaps": [], + "name": "return_one", + "output_def_snaps": [ { - "__class__": "InputDefSnap", - "dagster_type_key": "Nothing", + "__class__": "OutputDefSnap", + "dagster_type_key": "Int", "description": null, - "name": "start" + "is_dynamic": false, + "is_required": true, + "name": "result" } ], - "name": "no_output", - "output_def_snaps": [], "required_resource_keys": [], "tags": {} }, @@ -16987,16 +16740,22 @@ "input_def_snaps": [ { "__class__": "InputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "Int", "description": null, - "name": "value" + "name": "input_one" + }, + { + "__class__": "InputDefSnap", + "dagster_type_key": "Int", + "description": null, + "name": "input_two" } ], - "name": "passthrough", + "name": "sum_inputs", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "Int", "description": null, "is_dynamic": false, "is_required": true, @@ -17013,7 +16772,7 @@ ''' # --- # name: test_all_snapshot_ids[123] - '9a9a09da6b3a1dbab13bfde5e155c0f097dd22a5' + 'b07b89c1664d4a248bc1a39974a91b31d8956c54' # --- # name: test_all_snapshot_ids[124] ''' @@ -17436,6 +17195,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "fail", + "type_key": "Bool" + } + ], + "given_name": null, + "key": "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { "__class__": "ConfigTypeSnap", "description": null, @@ -17459,39 +17241,30 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "[DEPRECATED]", - "is_required": false, - "name": "marker_to_close", - "type_key": "String" - }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4678ad2ed1f8478f0abd5e4134850ab395b48f25": { + "Shape.381bcdb53d3182896a9a86cdfc9f625f62080106": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -17499,33 +17272,74 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "a", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "b", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Configure runtime parameters for ops or assets.", + "is_required": true, + "name": "ops", + "type_key": "Shape.857c24c238fce20e7bf57b59726c416b42738942" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.4678ad2ed1f8478f0abd5e4134850ab395b48f25", + "key": "Shape.381bcdb53d3182896a9a86cdfc9f625f62080106", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", + "is_required": false, + "name": "marker_to_close", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", + "is_required": false, + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + } + ], + "given_name": null, + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -17578,18 +17392,27 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.771375a24eca0dc65af691c7fe224aef2fb063e4": { + "Shape.857c24c238fce20e7bf57b59726c416b42738942": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "can_fail", + "type_key": "Shape.a62c224a9a675b7b35544be496fa9f28c429c6b6" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "start", + "name": "child_fail", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { @@ -17598,62 +17421,62 @@ "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "will_fail", + "name": "child_multi_skip", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.771375a24eca0dc65af691c7fe224aef2fb063e4", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.9e28720a11f1ded9af7902a3cd5503f7019018d2": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ + }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + "name": "child_skip", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", + "description": null, "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + "name": "grandchild_fail", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"start\": {}, \"will_fail\": {}}", - "description": "Configure runtime parameters for ops or assets.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "ops", - "type_key": "Shape.771375a24eca0dc65af691c7fe224aef2fb063e4" - }, + "name": "multi", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.857c24c238fce20e7bf57b59726c416b42738942", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.a62c224a9a675b7b35544be496fa9f28c429c6b6": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"a\": {}, \"b\": {}, \"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.4678ad2ed1f8478f0abd5e4134850ab395b48f25" + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9" } ], "given_name": null, - "key": "Shape.9e28720a11f1ded9af7902a3cd5503f7019018d2", + "key": "Shape.a62c224a9a675b7b35544be496fa9f28c429c6b6", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -17807,10 +17630,23 @@ "solid_invocation_snaps": [ { "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "inp", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "success", + "solid_name": "multi" + } + ] + } + ], "is_dynamic_mapped": false, - "solid_def_name": "start", - "solid_name": "start", + "solid_def_name": "can_fail", + "solid_name": "can_fail", "tags": {} }, { @@ -17818,26 +17654,97 @@ "input_dep_snaps": [ { "__class__": "InputDependencySnap", - "input_name": "num", + "input_name": "value", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "start_fail", + "solid_name": "can_fail" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "passthrough", + "solid_name": "child_fail", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "start", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "skip", + "solid_name": "multi" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "no_output", + "solid_name": "child_multi_skip", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "start", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "start_skip", + "solid_name": "can_fail" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "no_output", + "solid_name": "child_skip", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "start", "is_dynamic_collect": false, "upstream_output_snaps": [ { "__class__": "OutputHandleSnap", "output_name": "result", - "solid_name": "start" + "solid_name": "child_fail" } ] } ], "is_dynamic_mapped": false, - "solid_def_name": "will_fail", - "solid_name": "will_fail", + "solid_def_name": "no_output", + "solid_name": "grandchild_fail", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "multi", + "solid_name": "multi", "tags": {} } ] }, "description": null, - "graph_def_name": "retry_resource_job", + "graph_def_name": "retry_multi_output_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -17861,34 +17768,6 @@ ], "name": "default", "resource_def_snaps": [ - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "name": "a" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "name": "b" - }, { "__class__": "ResourceDefSnap", "config_field_snap": { @@ -17904,14 +17783,56 @@ "name": "io_manager" } ], - "root_config_key": "Shape.9e28720a11f1ded9af7902a3cd5503f7019018d2" + "root_config_key": "Shape.381bcdb53d3182896a9a86cdfc9f625f62080106" } ], - "name": "retry_resource_job", + "name": "retry_multi_output_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.0f759441a0a5dcbc16a099b6d7d106319ac492f9" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "String", + "description": null, + "name": "inp" + } + ], + "name": "can_fail", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "String", + "description": null, + "is_dynamic": false, + "is_required": false, + "name": "start_fail" + }, + { + "__class__": "OutputDefSnap", + "dagster_type_key": "String", + "description": null, + "is_dynamic": false, + "is_required": false, + "name": "start_skip" + } + ], + "required_resource_keys": [], + "tags": {} + }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -17925,20 +17846,51 @@ }, "description": null, "input_def_snaps": [], - "name": "start", + "name": "multi", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "String", "description": null, "is_dynamic": false, - "is_required": true, - "name": "result" + "is_required": false, + "name": "success" + }, + { + "__class__": "OutputDefSnap", + "dagster_type_key": "String", + "description": null, + "is_dynamic": false, + "is_required": false, + "name": "skip" } ], - "required_resource_keys": [ - "a" + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Nothing", + "description": null, + "name": "start" + } ], + "name": "no_output", + "output_def_snaps": [], + "required_resource_keys": [], "tags": {} }, { @@ -17958,10 +17910,10 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "num" + "name": "value" } ], - "name": "will_fail", + "name": "passthrough", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -17972,9 +17924,7 @@ "name": "result" } ], - "required_resource_keys": [ - "b" - ], + "required_resource_keys": [], "tags": {} } ] @@ -17984,7 +17934,7 @@ ''' # --- # name: test_all_snapshot_ids[125] - 'b131f0bc1872f377636fbf002a3ad86049c1e8af' + '9a9a09da6b3a1dbab13bfde5e155c0f097dd22a5' # --- # name: test_all_snapshot_ids[126] ''' @@ -18430,29 +18380,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, @@ -18485,50 +18412,41 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.476ef6b206140f25df8424fbc5a6d110fa109b3c": { + "Shape.4678ad2ed1f8478f0abd5e4134850ab395b48f25": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", + "description": null, "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + "name": "a", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"return_any\": {}, \"return_bool\": {}, \"return_int\": {}, \"return_str\": {}}", - "description": "Configure runtime parameters for ops or assets.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "ops", - "type_key": "Shape.b47c9be4e0b0e4112dc017682fccd4bfb071e216" + "name": "b", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.476ef6b206140f25df8424fbc5a6d110fa109b3c", + "key": "Shape.4678ad2ed1f8478f0abd5e4134850ab395b48f25", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -18581,7 +18499,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.b47c9be4e0b0e4112dc017682fccd4bfb071e216": { + "Shape.771375a24eca0dc65af691c7fe224aef2fb063e4": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -18592,7 +18510,7 @@ "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "return_any", + "name": "start", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { @@ -18601,30 +18519,62 @@ "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "return_bool", + "name": "will_fail", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.771375a24eca0dc65af691c7fe224aef2fb063e4", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.9e28720a11f1ded9af7902a3cd5503f7019018d2": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "return_int", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"start\": {}, \"will_fail\": {}}", + "description": "Configure runtime parameters for ops or assets.", "is_required": false, - "name": "return_str", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "ops", + "type_key": "Shape.771375a24eca0dc65af691c7fe224aef2fb063e4" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"a\": {}, \"b\": {}, \"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.4678ad2ed1f8478f0abd5e4134850ab395b48f25" } ], "given_name": null, - "key": "Shape.b47c9be4e0b0e4112dc017682fccd4bfb071e216", + "key": "Shape.9e28720a11f1ded9af7902a3cd5503f7019018d2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -18780,38 +18730,35 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "return_any", - "solid_name": "return_any", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "return_bool", - "solid_name": "return_bool", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "return_int", - "solid_name": "return_int", + "solid_def_name": "start", + "solid_name": "start", "tags": {} }, { "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "num", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "start" + } + ] + } + ], "is_dynamic_mapped": false, - "solid_def_name": "return_str", - "solid_name": "return_str", + "solid_def_name": "will_fail", + "solid_name": "will_fail", "tags": {} } ] }, "description": null, - "graph_def_name": "scalar_output_job", + "graph_def_name": "retry_resource_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -18835,6 +18782,34 @@ ], "name": "default", "resource_def_snaps": [ + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "name": "a" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "name": "b" + }, { "__class__": "ResourceDefSnap", "config_field_snap": { @@ -18850,10 +18825,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.476ef6b206140f25df8424fbc5a6d110fa109b3c" + "root_config_key": "Shape.9e28720a11f1ded9af7902a3cd5503f7019018d2" } ], - "name": "scalar_output_job", + "name": "retry_resource_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -18871,7 +18846,7 @@ }, "description": null, "input_def_snaps": [], - "name": "return_any", + "name": "start", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -18882,34 +18857,9 @@ "name": "result" } ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "return_bool", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Bool", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } + "required_resource_keys": [ + "a" ], - "required_resource_keys": [], "tags": {} }, { @@ -18924,46 +18874,28 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "return_int", - "output_def_snaps": [ + "input_def_snaps": [ { - "__class__": "OutputDefSnap", - "dagster_type_key": "Int", + "__class__": "InputDefSnap", + "dagster_type_key": "Any", "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" + "name": "num" } ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "return_str", + "name": "will_fail", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "String", + "dagster_type_key": "Any", "description": null, "is_dynamic": false, "is_required": true, "name": "result" } ], - "required_resource_keys": [], + "required_resource_keys": [ + "b" + ], "tags": {} } ] @@ -18973,7 +18905,7 @@ ''' # --- # name: test_all_snapshot_ids[127] - 'ca68b52613bfcadc230b3dc1ec7e1045f9bc5836' + 'b131f0bc1872f377636fbf002a3ad86049c1e8af' # --- # name: test_all_snapshot_ids[128] ''' @@ -19442,79 +19374,995 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"noop_op\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.242592fa9f0be8d5908506e918e119be06358618" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" - } - ], - "given_name": null, - "key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.242592fa9f0be8d5908506e918e119be06358618": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "noop_op", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.242592fa9f0be8d5908506e918e119be06358618", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", + "is_required": false, + "name": "marker_to_close", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", + "is_required": false, + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + } + ], + "given_name": null, + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.476ef6b206140f25df8424fbc5a6d110fa109b3c": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"return_any\": {}, \"return_bool\": {}, \"return_int\": {}, \"return_str\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.b47c9be4e0b0e4112dc017682fccd4bfb071e216" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + } + ], + "given_name": null, + "key": "Shape.476ef6b206140f25df8424fbc5a6d110fa109b3c", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "path", + "type_key": "String" + } + ], + "given_name": null, + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.b47c9be4e0b0e4112dc017682fccd4bfb071e216": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "return_any", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "return_bool", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "return_int", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "return_str", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.b47c9be4e0b0e4112dc017682fccd4bfb071e216", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [], + "given_name": null, + "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "console", + "type_key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a" + } + ], + "given_name": null, + "key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "String": { + "__class__": "ConfigTypeSnap", + "description": "", + "enum_values": null, + "fields": null, + "given_name": "String", + "key": "String", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR" + }, + "scalar_kind": { + "__enum__": "ConfigScalarKind.STRING" + }, + "type_param_keys": null + } + } + }, + "dagster_type_namespace_snapshot": { + "__class__": "DagsterTypeNamespaceSnapshot", + "all_dagster_type_snaps_by_key": { + "Any": { + "__class__": "DagsterTypeSnap", + "description": null, + "display_name": "Any", + "is_builtin": true, + "key": "Any", + "kind": { + "__enum__": "DagsterTypeKind.ANY" + }, + "loader_schema_key": "Selector.f2fe6dfdc60a1947a8f8e7cd377a012b47065bc4", + "materializer_schema_key": null, + "name": "Any", + "type_param_keys": [] + }, + "Bool": { + "__class__": "DagsterTypeSnap", + "description": null, + "display_name": "Bool", + "is_builtin": true, + "key": "Bool", + "kind": { + "__enum__": "DagsterTypeKind.SCALAR" + }, + "loader_schema_key": "ScalarUnion.Bool-Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59", + "materializer_schema_key": null, + "name": "Bool", + "type_param_keys": [] + }, + "Float": { + "__class__": "DagsterTypeSnap", + "description": null, + "display_name": "Float", + "is_builtin": true, + "key": "Float", + "kind": { + "__enum__": "DagsterTypeKind.SCALAR" + }, + "loader_schema_key": "ScalarUnion.Float-Selector.d00a37e3807d37c9f69cc62997c4a5f4a176e5c3", + "materializer_schema_key": null, + "name": "Float", + "type_param_keys": [] + }, + "Int": { + "__class__": "DagsterTypeSnap", + "description": null, + "display_name": "Int", + "is_builtin": true, + "key": "Int", + "kind": { + "__enum__": "DagsterTypeKind.SCALAR" + }, + "loader_schema_key": "ScalarUnion.Int-Selector.a9799b971d12ace70a2d8803c883c863417d0725", + "materializer_schema_key": null, + "name": "Int", + "type_param_keys": [] + }, + "Nothing": { + "__class__": "DagsterTypeSnap", + "description": null, + "display_name": "Nothing", + "is_builtin": true, + "key": "Nothing", + "kind": { + "__enum__": "DagsterTypeKind.NOTHING" + }, + "loader_schema_key": null, + "materializer_schema_key": null, + "name": "Nothing", + "type_param_keys": [] + }, + "String": { + "__class__": "DagsterTypeSnap", + "description": null, + "display_name": "String", + "is_builtin": true, + "key": "String", + "kind": { + "__enum__": "DagsterTypeKind.SCALAR" + }, + "loader_schema_key": "ScalarUnion.String-Selector.e04723c9d9937e3ab21206435b22247cfbe58269", + "materializer_schema_key": null, + "name": "String", + "type_param_keys": [] + } + } + }, + "dep_structure_snapshot": { + "__class__": "DependencyStructureSnapshot", + "solid_invocation_snaps": [ + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "return_any", + "solid_name": "return_any", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "return_bool", + "solid_name": "return_bool", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "return_int", + "solid_name": "return_int", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "return_str", + "solid_name": "return_str", + "tags": {} + } + ] + }, + "description": null, + "graph_def_name": "scalar_output_job", + "lineage_snapshot": null, + "mode_def_snaps": [ + { + "__class__": "ModeDefSnap", + "description": null, + "logger_def_snaps": [ + { + "__class__": "LoggerDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", + "description": "The default colored console logger.", + "is_required": false, + "name": "config", + "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" + }, + "description": "The default colored console logger.", + "name": "console" + } + ], + "name": "default", + "resource_def_snaps": [ + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "name": "io_manager" + } + ], + "root_config_key": "Shape.476ef6b206140f25df8424fbc5a6d110fa109b3c" + } + ], + "name": "scalar_output_job", + "solid_definitions_snapshot": { + "__class__": "SolidDefinitionsSnapshot", + "composite_solid_def_snaps": [], + "solid_def_snaps": [ + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "return_any", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "return_bool", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Bool", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "return_int", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Int", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "return_str", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "String", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + } + ] + }, + "tags": {} + } + ''' +# --- +# name: test_all_snapshot_ids[129] + 'ca68b52613bfcadc230b3dc1ec7e1045f9bc5836' +# --- +# name: test_all_snapshot_ids[12] + ''' + { + "__class__": "PipelineSnapshot", + "config_schema_snapshot": { + "__class__": "ConfigSchemaSnapshot", + "all_config_snaps_by_key": { + "Any": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": "Any", + "key": "Any", + "kind": { + "__enum__": "ConfigTypeKind.ANY" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Bool": { + "__class__": "ConfigTypeSnap", + "description": "", + "enum_values": null, + "fields": null, + "given_name": "Bool", + "key": "Bool", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR" + }, + "scalar_kind": { + "__enum__": "ConfigScalarKind.BOOL" + }, + "type_param_keys": null + }, + "Float": { + "__class__": "ConfigTypeSnap", + "description": "", + "enum_values": null, + "fields": null, + "given_name": "Float", + "key": "Float", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR" + }, + "scalar_kind": { + "__enum__": "ConfigScalarKind.FLOAT" + }, + "type_param_keys": null + }, + "Int": { + "__class__": "ConfigTypeSnap", + "description": "", + "enum_values": null, + "fields": null, + "given_name": "Int", + "key": "Int", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR" + }, + "scalar_kind": { + "__enum__": "ConfigScalarKind.INT" + }, + "type_param_keys": null + }, + "ScalarUnion.Bool-Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "ScalarUnion.Bool-Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "Bool", + "Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59" + ] + }, + "ScalarUnion.Float-Selector.d00a37e3807d37c9f69cc62997c4a5f4a176e5c3": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "ScalarUnion.Float-Selector.d00a37e3807d37c9f69cc62997c4a5f4a176e5c3", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "Float", + "Selector.d00a37e3807d37c9f69cc62997c4a5f4a176e5c3" + ] + }, + "ScalarUnion.Int-Selector.a9799b971d12ace70a2d8803c883c863417d0725": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "ScalarUnion.Int-Selector.a9799b971d12ace70a2d8803c883c863417d0725", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "Int", + "Selector.a9799b971d12ace70a2d8803c883c863417d0725" + ] + }, + "ScalarUnion.String-Selector.e04723c9d9937e3ab21206435b22247cfbe58269": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "ScalarUnion.String-Selector.e04723c9d9937e3ab21206435b22247cfbe58269", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "String", + "Selector.e04723c9d9937e3ab21206435b22247cfbe58269" + ] + }, + "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "disabled", + "type_key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "enabled", + "type_key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709" + } + ], + "given_name": null, + "key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "json", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "pickle", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "value", + "type_key": "Int" + } + ], + "given_name": null, + "key": "Selector.a9799b971d12ace70a2d8803c883c863417d0725", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "json", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "pickle", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "value", + "type_key": "Bool" + } + ], + "given_name": null, + "key": "Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Selector.d00a37e3807d37c9f69cc62997c4a5f4a176e5c3": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "json", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "pickle", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "value", + "type_key": "Float" + } + ], + "given_name": null, + "key": "Selector.d00a37e3807d37c9f69cc62997c4a5f4a176e5c3", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Selector.e04723c9d9937e3ab21206435b22247cfbe58269": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "json", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "pickle", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "value", + "type_key": "String" + } + ], + "given_name": null, + "key": "Selector.e04723c9d9937e3ab21206435b22247cfbe58269", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Selector.f2fe6dfdc60a1947a8f8e7cd377a012b47065bc4": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "json", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "pickle", + "type_key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "value", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Selector.f2fe6dfdc60a1947a8f8e7cd377a012b47065bc4", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.081354663b9d4b8fbfd1cb8e358763912953913f": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "\"INFO\"", + "description": "The logger's threshold.", + "is_required": false, + "name": "log_level", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "\"dagster\"", + "description": "The name of your logger.", + "is_required": false, + "name": "name", + "type_key": "String" + } + ], + "given_name": null, + "key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.09d73f0755bf4752d3f121837669c8660dcf451e": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"retries\": {\"enabled\": {}}}", + "description": "Execute all steps in a single process.", + "is_required": false, + "name": "config", + "type_key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c" + } + ], + "given_name": null, + "key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", + "description": "The default colored console logger.", + "is_required": false, + "name": "config", + "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" + } + ], + "given_name": null, + "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, @@ -19570,6 +20418,47 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.571dd0dfd379812de73514675817becc7f384e4c": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "after_failure", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "always_succeed", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "conditionally_fail", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.571dd0dfd379812de73514675817becc7f384e4c", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, @@ -19606,6 +20495,56 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.e216ae17937e7ca0141e3e837ac6661142027def": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"after_failure\": {}, \"always_succeed\": {}, \"conditionally_fail\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.571dd0dfd379812de73514675817becc7f384e4c" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + } + ], + "given_name": null, + "key": "Shape.e216ae17937e7ca0141e3e837ac6661142027def", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { "__class__": "ConfigTypeSnap", "description": null, @@ -19738,18 +20677,60 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "_upstream", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "conditionally_fail" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "after_failure", + "solid_name": "after_failure", + "tags": {} + }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "noop_op", - "solid_name": "noop_op", + "solid_def_name": "always_succeed", + "solid_name": "always_succeed", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "_upstream", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "always_succeed" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "conditionally_fail", + "solid_name": "conditionally_fail", "tags": {} } ] }, "description": null, - "graph_def_name": "simple_graph", + "graph_def_name": "chained_failure_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -19788,14 +20769,48 @@ "name": "io_manager" } ], - "root_config_key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a" + "root_config_key": "Shape.e216ae17937e7ca0141e3e837ac6661142027def" } ], - "name": "simple_job_a", + "name": "chained_failure_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "_upstream" + } + ], + "name": "after_failure", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -19809,7 +20824,41 @@ }, "description": null, "input_def_snaps": [], - "name": "noop_op", + "name": "always_succeed", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "_upstream" + } + ], + "name": "conditionally_fail", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -19829,10 +20878,7 @@ } ''' # --- -# name: test_all_snapshot_ids[129] - '4fbe2b21985de715173a8d630c5d1506a0c7f040' -# --- -# name: test_all_snapshot_ids[12] +# name: test_all_snapshot_ids[130] ''' { "__class__": "PipelineSnapshot", @@ -20299,96 +21345,128 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "[DEPRECATED]", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "marker_to_close", - "type_key": "String" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"noop_op\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.242592fa9f0be8d5908506e918e119be06358618" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.242592fa9f0be8d5908506e918e119be06358618": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "path", - "type_key": "String" + "is_required": false, + "name": "noop_op", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.242592fa9f0be8d5908506e918e119be06358618", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.571dd0dfd379812de73514675817becc7f384e4c": { + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", "is_required": false, - "name": "after_failure", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "marker_to_close", + "type_key": "String" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", "is_required": false, - "name": "always_succeed", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + } + ], + "given_name": null, + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "conditionally_fail", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "is_required": true, + "name": "path", + "type_key": "String" } ], "given_name": null, - "key": "Shape.571dd0dfd379812de73514675817becc7f384e4c", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -20431,56 +21509,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.e216ae17937e7ca0141e3e837ac6661142027def": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"after_failure\": {}, \"always_succeed\": {}, \"conditionally_fail\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.571dd0dfd379812de73514675817becc7f384e4c" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" - } - ], - "given_name": null, - "key": "Shape.e216ae17937e7ca0141e3e837ac6661142027def", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { "__class__": "ConfigTypeSnap", "description": null, @@ -20613,60 +21641,18 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "_upstream", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "conditionally_fail" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "after_failure", - "solid_name": "after_failure", - "tags": {} - }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "always_succeed", - "solid_name": "always_succeed", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "_upstream", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "always_succeed" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "conditionally_fail", - "solid_name": "conditionally_fail", + "solid_def_name": "noop_op", + "solid_name": "noop_op", "tags": {} } ] }, "description": null, - "graph_def_name": "chained_failure_job", + "graph_def_name": "simple_graph", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -20705,48 +21691,14 @@ "name": "io_manager" } ], - "root_config_key": "Shape.e216ae17937e7ca0141e3e837ac6661142027def" + "root_config_key": "Shape.1baaa7ea6f13caf1cca874c6f4cc8581ad88d08a" } ], - "name": "chained_failure_job", + "name": "simple_job_a", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "_upstream" - } - ], - "name": "after_failure", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -20760,41 +21712,7 @@ }, "description": null, "input_def_snaps": [], - "name": "always_succeed", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "_upstream" - } - ], - "name": "conditionally_fail", + "name": "noop_op", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -20814,7 +21732,10 @@ } ''' # --- -# name: test_all_snapshot_ids[130] +# name: test_all_snapshot_ids[131] + '4fbe2b21985de715173a8d630c5d1506a0c7f040' +# --- +# name: test_all_snapshot_ids[132] ''' { "__class__": "PipelineSnapshot", @@ -21668,10 +22589,10 @@ } ''' # --- -# name: test_all_snapshot_ids[131] +# name: test_all_snapshot_ids[133] 'c58bbee5967b5e43b907f2ee09bfa26b5b017899' # --- -# name: test_all_snapshot_ids[132] +# name: test_all_snapshot_ids[134] ''' { "__class__": "PipelineSnapshot", @@ -22525,10 +23446,10 @@ } ''' # --- -# name: test_all_snapshot_ids[133] +# name: test_all_snapshot_ids[135] '9109ebc6447db70cdcd17a7bf8b90ec61ffd02fe' # --- -# name: test_all_snapshot_ids[134] +# name: test_all_snapshot_ids[136] ''' { "__class__": "PipelineSnapshot", @@ -23382,10 +24303,10 @@ } ''' # --- -# name: test_all_snapshot_ids[135] +# name: test_all_snapshot_ids[137] '673ec5c1c0fce5ffec0506edd32bcdbd018e79bb' # --- -# name: test_all_snapshot_ids[136] +# name: test_all_snapshot_ids[138] ''' { "__class__": "PipelineSnapshot", @@ -24641,10 +25562,13 @@ } ''' # --- -# name: test_all_snapshot_ids[137] +# name: test_all_snapshot_ids[139] 'ea8e78a81a6c470713834d21c023ab1d45d00394' # --- -# name: test_all_snapshot_ids[138] +# name: test_all_snapshot_ids[13] + '1f3478e419b57370edfc5959b967300b91ad776c' +# --- +# name: test_all_snapshot_ids[140] ''' { "__class__": "PipelineSnapshot", @@ -25498,13 +26422,10 @@ } ''' # --- -# name: test_all_snapshot_ids[139] +# name: test_all_snapshot_ids[141] '927cbfcff5af3bd40ebed1cae3eed1baf4c3547f' # --- -# name: test_all_snapshot_ids[13] - '1f3478e419b57370edfc5959b967300b91ad776c' -# --- -# name: test_all_snapshot_ids[140] +# name: test_all_snapshot_ids[142] ''' { "__class__": "PipelineSnapshot", @@ -26363,10 +27284,10 @@ } ''' # --- -# name: test_all_snapshot_ids[141] +# name: test_all_snapshot_ids[143] 'c3320c44d5fa6c508dfda564e67bc67747b9891c' # --- -# name: test_all_snapshot_ids[142] +# name: test_all_snapshot_ids[144] ''' { "__class__": "PipelineSnapshot", @@ -27462,10 +28383,10 @@ } ''' # --- -# name: test_all_snapshot_ids[143] +# name: test_all_snapshot_ids[145] '81ecba8b867d1179f0998c86b85a1176aa946456' # --- -# name: test_all_snapshot_ids[144] +# name: test_all_snapshot_ids[146] ''' { "__class__": "PipelineSnapshot", @@ -28561,10 +29482,10 @@ } ''' # --- -# name: test_all_snapshot_ids[145] +# name: test_all_snapshot_ids[147] 'ce839b23a887f29520c71238335886cbe80337ef' # --- -# name: test_all_snapshot_ids[146] +# name: test_all_snapshot_ids[148] ''' { "__class__": "PipelineSnapshot", @@ -29544,10 +30465,10 @@ } ''' # --- -# name: test_all_snapshot_ids[147] +# name: test_all_snapshot_ids[149] '4b4b18dca82ef0567492f476ff2bcbbf7392206d' # --- -# name: test_all_snapshot_ids[148] +# name: test_all_snapshot_ids[14] ''' { "__class__": "PipelineSnapshot", @@ -30138,47 +31059,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.7287607714eeb58d5c1cf75b5e10a5a62526a2f9": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "typed_asset", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": null, - "is_required": false, - "name": "typed_multi_asset", - "type_key": "Shape.62edccaf30696e25335ae92685bdc41e204e30e6" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "untyped_asset", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.7287607714eeb58d5c1cf75b5e10a5a62526a2f9", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, @@ -30289,7 +31169,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.cf22461b7e3701053d0f604828a7a7b537619104": { + "Shape.d308bf158c48be068392c81eb707562cc2d96158": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -30315,11 +31195,11 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"typed_asset\": {}, \"typed_multi_asset\": {\"config\": {}}, \"untyped_asset\": {}}", + "default_value_as_json_str": "{\"subsettable_checked_multi_asset\": {\"config\": {}}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.7287607714eeb58d5c1cf75b5e10a5a62526a2f9" + "type_key": "Shape.f6636d306fe2f9e3ee64a75e0c83bcfeda8f215b" }, { "__class__": "ConfigFieldSnap", @@ -30332,7 +31212,7 @@ } ], "given_name": null, - "key": "Shape.cf22461b7e3701053d0f604828a7a7b537619104", + "key": "Shape.d308bf158c48be068392c81eb707562cc2d96158", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -30375,6 +31255,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.f6636d306fe2f9e3ee64a75e0c83bcfeda8f215b": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {}}", + "description": null, + "is_required": false, + "name": "subsettable_checked_multi_asset", + "type_key": "Shape.62edccaf30696e25335ae92685bdc41e204e30e6" + } + ], + "given_name": null, + "key": "Shape.f6636d306fe2f9e3ee64a75e0c83bcfeda8f215b", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -30500,60 +31403,18 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "int_asset", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "int_asset", - "solid_name": "typed_multi_asset" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "typed_asset", - "solid_name": "typed_asset", - "tags": {} - }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "typed_multi_asset", - "solid_name": "typed_multi_asset", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "typed_asset", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "typed_asset" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "untyped_asset", - "solid_name": "untyped_asset", + "solid_def_name": "subsettable_checked_multi_asset", + "solid_name": "subsettable_checked_multi_asset", "tags": {} } ] }, "description": null, - "graph_def_name": "typed_assets", + "graph_def_name": "checked_multi_asset_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -30620,49 +31481,15 @@ "name": "io_manager" } ], - "root_config_key": "Shape.cf22461b7e3701053d0f604828a7a7b537619104" + "root_config_key": "Shape.d308bf158c48be068392c81eb707562cc2d96158" } ], - "name": "typed_assets", + "name": "checked_multi_asset_job", "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "int_asset" - } - ], - "name": "typed_asset", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Int", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -30676,57 +31503,39 @@ }, "description": null, "input_def_snaps": [], - "name": "typed_multi_asset", + "name": "subsettable_checked_multi_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Int", + "dagster_type_key": "Any", "description": null, "is_dynamic": false, - "is_required": true, - "name": "int_asset" + "is_required": false, + "name": "one" }, { "__class__": "OutputDefSnap", - "dagster_type_key": "String", + "dagster_type_key": "Any", "description": null, "is_dynamic": false, - "is_required": true, - "name": "str_asset" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ + "is_required": false, + "name": "two" + }, { - "__class__": "InputDefSnap", + "__class__": "OutputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "typed_asset" - } - ], - "name": "untyped_asset", - "output_def_snaps": [ + "is_dynamic": false, + "is_required": false, + "name": "one_my_check" + }, { "__class__": "OutputDefSnap", "dagster_type_key": "Any", "description": null, "is_dynamic": false, - "is_required": true, - "name": "result" + "is_required": false, + "name": "one_my_other_check" } ], "required_resource_keys": [], @@ -30738,10 +31547,7 @@ } ''' # --- -# name: test_all_snapshot_ids[149] - 'dedb9fc5a6627950d8d31dce78af887c18c25160' -# --- -# name: test_all_snapshot_ids[14] +# name: test_all_snapshot_ids[150] ''' { "__class__": "PipelineSnapshot", @@ -31332,6 +32138,47 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.7287607714eeb58d5c1cf75b5e10a5a62526a2f9": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "typed_asset", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {}}", + "description": null, + "is_required": false, + "name": "typed_multi_asset", + "type_key": "Shape.62edccaf30696e25335ae92685bdc41e204e30e6" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "untyped_asset", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.7287607714eeb58d5c1cf75b5e10a5a62526a2f9", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, @@ -31442,7 +32289,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.d308bf158c48be068392c81eb707562cc2d96158": { + "Shape.cf22461b7e3701053d0f604828a7a7b537619104": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -31468,11 +32315,11 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"subsettable_checked_multi_asset\": {\"config\": {}}}", + "default_value_as_json_str": "{\"typed_asset\": {}, \"typed_multi_asset\": {\"config\": {}}, \"untyped_asset\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.f6636d306fe2f9e3ee64a75e0c83bcfeda8f215b" + "type_key": "Shape.7287607714eeb58d5c1cf75b5e10a5a62526a2f9" }, { "__class__": "ConfigFieldSnap", @@ -31485,7 +32332,7 @@ } ], "given_name": null, - "key": "Shape.d308bf158c48be068392c81eb707562cc2d96158", + "key": "Shape.cf22461b7e3701053d0f604828a7a7b537619104", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -31528,29 +32375,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.f6636d306fe2f9e3ee64a75e0c83bcfeda8f215b": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": null, - "is_required": false, - "name": "subsettable_checked_multi_asset", - "type_key": "Shape.62edccaf30696e25335ae92685bdc41e204e30e6" - } - ], - "given_name": null, - "key": "Shape.f6636d306fe2f9e3ee64a75e0c83bcfeda8f215b", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -31676,18 +32500,60 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "int_asset", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "int_asset", + "solid_name": "typed_multi_asset" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "typed_asset", + "solid_name": "typed_asset", + "tags": {} + }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "subsettable_checked_multi_asset", - "solid_name": "subsettable_checked_multi_asset", + "solid_def_name": "typed_multi_asset", + "solid_name": "typed_multi_asset", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "typed_asset", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "typed_asset" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "untyped_asset", + "solid_name": "untyped_asset", "tags": {} } ] }, "description": null, - "graph_def_name": "checked_multi_asset_job", + "graph_def_name": "typed_assets", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -31754,15 +32620,49 @@ "name": "io_manager" } ], - "root_config_key": "Shape.d308bf158c48be068392c81eb707562cc2d96158" + "root_config_key": "Shape.cf22461b7e3701053d0f604828a7a7b537619104" } ], - "name": "checked_multi_asset_job", + "name": "typed_assets", "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "int_asset" + } + ], + "name": "typed_asset", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Int", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -31776,39 +32676,57 @@ }, "description": null, "input_def_snaps": [], - "name": "subsettable_checked_multi_asset", + "name": "typed_multi_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "Int", "description": null, "is_dynamic": false, - "is_required": false, - "name": "one" + "is_required": true, + "name": "int_asset" }, { "__class__": "OutputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "String", "description": null, "is_dynamic": false, - "is_required": false, - "name": "two" - }, + "is_required": true, + "name": "str_asset" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ { - "__class__": "OutputDefSnap", + "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "is_dynamic": false, - "is_required": false, - "name": "one_my_check" - }, + "name": "typed_asset" + } + ], + "name": "untyped_asset", + "output_def_snaps": [ { "__class__": "OutputDefSnap", "dagster_type_key": "Any", "description": null, "is_dynamic": false, - "is_required": false, - "name": "one_my_other_check" + "is_required": true, + "name": "result" } ], "required_resource_keys": [], @@ -31820,6 +32738,9 @@ } ''' # --- +# name: test_all_snapshot_ids[151] + 'dedb9fc5a6627950d8d31dce78af887c18c25160' +# --- # name: test_all_snapshot_ids[15] 'c77559c3ffec162d2580606b6cc7aeaf8ab8d9e0' # --- @@ -56428,94 +57349,94 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", + "is_required": false, + "name": "marker_to_close", + "type_key": "String" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.369ed0636f08f329e2aa8b098ffa6a79fc232dcf": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"fail\": false}}", - "description": null, - "is_required": false, - "name": "hard_fail_or_0", - "type_key": "Shape.bcef613ddd9bdac33326b7d43d149dbf594a763c" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "increment", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "is_required": true, + "name": "path", + "type_key": "String" } ], "given_name": null, - "key": "Shape.369ed0636f08f329e2aa8b098ffa6a79fc232dcf", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.6732c397659f3b19ac9e39a3531071679cd0f5dc": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "[DEPRECATED]", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "marker_to_close", - "type_key": "String" + "name": "hanging_op", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + "name": "my_op", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.6732c397659f3b19ac9e39a3531071679cd0f5dc", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -56525,132 +57446,141 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "path", - "type_key": "String" + "is_required": false, + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.9409c5c85d96cec6085406c6dce69f398f5f3497": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"hanging_op\": {}, \"my_op\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.6732c397659f3b19ac9e39a3531071679cd0f5dc" + }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" + "description": "Configure how shared resources are implemented within a run.", + "is_required": true, + "name": "resources", + "type_key": "Shape.b2a98b3695b3c5a5db30815c530d115695019f1d" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.9409c5c85d96cec6085406c6dce69f398f5f3497", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.a269e234e7f0491d780456ee5305fd928c183afa": { + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "false", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "fail", - "type_key": "Bool" + "is_required": true, + "name": "file", + "type_key": "String" } ], "given_name": null, - "key": "Shape.a269e234e7f0491d780456ee5305fd928c183afa", + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.bcef613ddd9bdac33326b7d43d149dbf594a763c": { + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"fail\": false}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Shape.a269e234e7f0491d780456ee5305fd928c183afa" + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" } ], "given_name": null, - "key": "Shape.bcef613ddd9bdac33326b7d43d149dbf594a763c", + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.c1ce722407ab5e802d58b8bebd0bc8c3eb4dfcbe": { + "Shape.b2a98b3695b3c5a5db30815c530d115695019f1d": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"hard_fail_or_0\": {\"config\": {\"fail\": false}}, \"increment\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.369ed0636f08f329e2aa8b098ffa6a79fc232dcf" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.c1ce722407ab5e802d58b8bebd0bc8c3eb4dfcbe", + "key": "Shape.b2a98b3695b3c5a5db30815c530d115695019f1d", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -56802,39 +57732,39 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "hard_fail_or_0", - "solid_name": "hard_fail_or_0", - "tags": {} - }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [ { "__class__": "InputDependencySnap", - "input_name": "n", + "input_name": "my_op", "is_dynamic_collect": false, "upstream_output_snaps": [ { "__class__": "OutputHandleSnap", "output_name": "result", - "solid_name": "hard_fail_or_0" + "solid_name": "my_op" } ] } ], "is_dynamic_mapped": false, - "solid_def_name": "increment", - "solid_name": "increment", + "solid_def_name": "hanging_op", + "solid_name": "hanging_op", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "my_op", + "solid_name": "my_op", "tags": {} } ] }, "description": null, - "graph_def_name": "hard_failer", + "graph_def_name": "hanging_partitioned_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -56858,6 +57788,20 @@ ], "name": "default", "resource_def_snaps": [ + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + }, + "description": null, + "name": "hanging_asset_resource" + }, { "__class__": "ResourceDefSnap", "config_field_snap": { @@ -56873,10 +57817,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.c1ce722407ab5e802d58b8bebd0bc8c3eb4dfcbe" + "root_config_key": "Shape.9409c5c85d96cec6085406c6dce69f398f5f3497" } ], - "name": "hard_failer", + "name": "hanging_partitioned_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -56885,27 +57829,36 @@ "__class__": "SolidDefSnap", "config_field_snap": { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"fail\": false}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, "name": "config", - "type_key": "Shape.a269e234e7f0491d780456ee5305fd928c183afa" + "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "hard_fail_or_0", + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "my_op" + } + ], + "name": "hanging_op", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Int", + "dagster_type_key": "Any", "description": null, "is_dynamic": false, "is_required": true, "name": "result" } ], - "required_resource_keys": [], + "required_resource_keys": [ + "hanging_asset_resource" + ], "tags": {} }, { @@ -56920,19 +57873,12 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "n" - } - ], - "name": "increment", + "input_def_snaps": [], + "name": "my_op", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Int", + "dagster_type_key": "Any", "description": null, "is_dynamic": false, "is_required": true, @@ -56949,7 +57895,7 @@ ''' # --- # name: test_all_snapshot_ids[57] - 'c6b504611b1ee7582092807ac90bdd4ac64bac3b' + 'b467f29b8fb76fd793f23aa1cefb10918a105f91' # --- # name: test_all_snapshot_ids[58] ''' @@ -57418,6 +58364,38 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.369ed0636f08f329e2aa8b098ffa6a79fc232dcf": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"fail\": false}}", + "description": null, + "is_required": false, + "name": "hard_fail_or_0", + "type_key": "Shape.bcef613ddd9bdac33326b7d43d149dbf594a763c" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "increment", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.369ed0636f08f329e2aa8b098ffa6a79fc232dcf", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, @@ -57473,7 +58451,30 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.5b0100a2f69412eeae1c1ce42684400e4eac9903": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.a269e234e7f0491d780456ee5305fd928c183afa": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -57481,45 +58482,45 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", + "default_value_as_json_str": "false", "description": null, "is_required": false, - "name": "op_that_gets_tags", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "fail", + "type_key": "Bool" } ], "given_name": null, - "key": "Shape.5b0100a2f69412eeae1c1ce42684400e4eac9903", + "key": "Shape.a269e234e7f0491d780456ee5305fd928c183afa", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.bcef613ddd9bdac33326b7d43d149dbf594a763c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{\"fail\": false}", "description": null, "is_required": false, "name": "config", - "type_key": "Any" + "type_key": "Shape.a269e234e7f0491d780456ee5305fd928c183afa" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.bcef613ddd9bdac33326b7d43d149dbf594a763c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.89a658879dc08885cdbc7c5fffe10bb1fcd55d81": { + "Shape.c1ce722407ab5e802d58b8bebd0bc8c3eb4dfcbe": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -57545,11 +58546,11 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"op_that_gets_tags\": {}}", + "default_value_as_json_str": "{\"hard_fail_or_0\": {\"config\": {\"fail\": false}}, \"increment\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.5b0100a2f69412eeae1c1ce42684400e4eac9903" + "type_key": "Shape.369ed0636f08f329e2aa8b098ffa6a79fc232dcf" }, { "__class__": "ConfigFieldSnap", @@ -57562,7 +58563,7 @@ } ], "given_name": null, - "key": "Shape.89a658879dc08885cdbc7c5fffe10bb1fcd55d81", + "key": "Shape.c1ce722407ab5e802d58b8bebd0bc8c3eb4dfcbe", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -57718,14 +58719,35 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_that_gets_tags", - "solid_name": "op_that_gets_tags", + "solid_def_name": "hard_fail_or_0", + "solid_name": "hard_fail_or_0", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "n", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "hard_fail_or_0" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "increment", + "solid_name": "increment", "tags": {} } ] }, "description": null, - "graph_def_name": "hello_world_with_tags", + "graph_def_name": "hard_failer", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -57764,14 +58786,41 @@ "name": "io_manager" } ], - "root_config_key": "Shape.89a658879dc08885cdbc7c5fffe10bb1fcd55d81" + "root_config_key": "Shape.c1ce722407ab5e802d58b8bebd0bc8c3eb4dfcbe" } ], - "name": "hello_world_with_tags", + "name": "hard_failer", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"fail\": false}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.a269e234e7f0491d780456ee5305fd928c183afa" + }, + "description": null, + "input_def_snaps": [], + "name": "hard_fail_or_0", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Int", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -57784,12 +58833,19 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "op_that_gets_tags", + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Int", + "description": null, + "name": "n" + } + ], + "name": "increment", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "Int", "description": null, "is_dynamic": false, "is_required": true, @@ -57801,14 +58857,12 @@ } ] }, - "tags": { - "tag_key": "tag_value" - } + "tags": {} } ''' # --- # name: test_all_snapshot_ids[59] - 'dc190868e8887ba5ac3669da13473252cd0ab098' + 'c6b504611b1ee7582092807ac90bdd4ac64bac3b' # --- # name: test_all_snapshot_ids[5] 'c23e75a6d1d7a9b89528399a9d27bcc9c90a178e' @@ -58335,6 +59389,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.5b0100a2f69412eeae1c1ce42684400e4eac9903": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "op_that_gets_tags", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.5b0100a2f69412eeae1c1ce42684400e4eac9903", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, @@ -58358,7 +59435,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.7a2485929d7236949338d36f0435c94a9e354a77": { + "Shape.89a658879dc08885cdbc7c5fffe10bb1fcd55d81": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -58383,12 +59460,12 @@ }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{\"op_that_gets_tags\": {}}", "description": "Configure runtime parameters for ops or assets.", - "is_required": true, + "is_required": false, "name": "ops", - "type_key": "Shape.f1d05596db6e35312901196d530dfee6946a33fe" + "type_key": "Shape.5b0100a2f69412eeae1c1ce42684400e4eac9903" }, { "__class__": "ConfigFieldSnap", @@ -58401,53 +59478,7 @@ } ], "given_name": null, - "key": "Shape.7a2485929d7236949338d36f0435c94a9e354a77", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "file", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" - } - ], - "given_name": null, - "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", + "key": "Shape.89a658879dc08885cdbc7c5fffe10bb1fcd55d81", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -58490,29 +59521,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.f1d05596db6e35312901196d530dfee6946a33fe": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "loop", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" - } - ], - "given_name": null, - "key": "Shape.f1d05596db6e35312901196d530dfee6946a33fe", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -58626,14 +59634,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "loop", - "solid_name": "loop", + "solid_def_name": "op_that_gets_tags", + "solid_name": "op_that_gets_tags", "tags": {} } ] }, "description": null, - "graph_def_name": "infinite_loop_job", + "graph_def_name": "hello_world_with_tags", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -58672,10 +59680,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.7a2485929d7236949338d36f0435c94a9e354a77" + "root_config_key": "Shape.89a658879dc08885cdbc7c5fffe10bb1fcd55d81" } ], - "name": "infinite_loop_job", + "name": "hello_world_with_tags", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -58687,13 +59695,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, + "is_required": false, "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + "type_key": "Any" }, "description": null, "input_def_snaps": [], - "name": "loop", + "name": "op_that_gets_tags", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -58709,12 +59717,14 @@ } ] }, - "tags": {} + "tags": { + "tag_key": "tag_value" + } } ''' # --- # name: test_all_snapshot_ids[61] - '83469e8c700778e0c1e268f158672fca54d5896b' + 'dc190868e8887ba5ac3669da13473252cd0ab098' # --- # name: test_all_snapshot_ids[62] ''' @@ -59238,7 +60248,30 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.61678299d213abc6dacb4a403face86263e676d7": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.7a2485929d7236949338d36f0435c94a9e354a77": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -59263,12 +60296,12 @@ }, { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"return_integer\": {}}", + "default_provided": false, + "default_value_as_json_str": null, "description": "Configure runtime parameters for ops or assets.", - "is_required": false, + "is_required": true, "name": "ops", - "type_key": "Shape.b237e55672e14459eb2f71b6889258a51aa88c86" + "type_key": "Shape.f1d05596db6e35312901196d530dfee6946a33fe" }, { "__class__": "ConfigFieldSnap", @@ -59281,14 +60314,14 @@ } ], "given_name": null, - "key": "Shape.61678299d213abc6dacb4a403face86263e676d7", + "key": "Shape.7a2485929d7236949338d36f0435c94a9e354a77", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -59298,36 +60331,36 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" + "is_required": true, + "name": "file", + "type_key": "String" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.b237e55672e14459eb2f71b6889258a51aa88c86": { + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "return_integer", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" } ], "given_name": null, - "key": "Shape.b237e55672e14459eb2f71b6889258a51aa88c86", + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -59370,6 +60403,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.f1d05596db6e35312901196d530dfee6946a33fe": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "loop", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + } + ], + "given_name": null, + "key": "Shape.f1d05596db6e35312901196d530dfee6946a33fe", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -59483,14 +60539,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "return_integer", - "solid_name": "return_integer", + "solid_def_name": "loop", + "solid_name": "loop", "tags": {} } ] }, "description": null, - "graph_def_name": "integers", + "graph_def_name": "infinite_loop_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -59529,10 +60585,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.61678299d213abc6dacb4a403face86263e676d7" + "root_config_key": "Shape.7a2485929d7236949338d36f0435c94a9e354a77" } ], - "name": "integers", + "name": "infinite_loop_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -59544,13 +60600,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" }, "description": null, "input_def_snaps": [], - "name": "return_integer", + "name": "loop", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -59571,7 +60627,7 @@ ''' # --- # name: test_all_snapshot_ids[63] - 'c4fee3a0c56b1ef6fcea9b64ceaacd06c4d5e216' + '83469e8c700778e0c1e268f158672fca54d5896b' # --- # name: test_all_snapshot_ids[64] ''' @@ -59734,29 +60790,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Selector.2571019f1a5201853d11032145ac3e534067f214": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "env", - "type_key": "String" - } - ], - "given_name": null, - "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", - "kind": { - "__enum__": "ConfigTypeKind.SELECTOR" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -60040,80 +61073,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "base_dir", - "type_key": "StringSourceType" - } - ], - "given_name": null, - "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.370e26b257ba7c336c1b21a7bd8e340ea17d2bc2": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"integers_asset\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.431e85768ac5c1cb79e6f0893ca4a2e5dda4d4a3" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" - } - ], - "given_name": null, - "key": "Shape.370e26b257ba7c336c1b21a7bd8e340ea17d2bc2", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.431e85768ac5c1cb79e6f0893ca4a2e5dda4d4a3": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -60122,14 +61082,14 @@ "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "integers_asset", + "name": "io_manager", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.431e85768ac5c1cb79e6f0893ca4a2e5dda4d4a3", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -60168,29 +61128,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - } - ], - "given_name": null, - "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, @@ -60214,71 +61151,57 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.61678299d213abc6dacb4a403face86263e676d7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "config", - "type_key": "Any" - } - ], - "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "dummy_io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{\"return_integer\": {}}", + "description": "Configure runtime parameters for ops or assets.", "is_required": false, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + "name": "ops", + "type_key": "Shape.b237e55672e14459eb2f71b6889258a51aa88c86" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", + "key": "Shape.61678299d213abc6dacb4a403face86263e676d7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -60288,36 +61211,36 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "file", - "type_key": "String" + "is_required": false, + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { + "Shape.b237e55672e14459eb2f71b6889258a51aa88c86": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + "is_required": false, + "name": "return_integer", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", + "key": "Shape.b237e55672e14459eb2f71b6889258a51aa88c86", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -60374,22 +61297,6 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null - }, - "StringSourceType": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": null, - "given_name": null, - "key": "StringSourceType", - "kind": { - "__enum__": "ConfigTypeKind.SCALAR_UNION" - }, - "scalar_kind": null, - "type_param_keys": [ - "String", - "Selector.2571019f1a5201853d11032145ac3e534067f214" - ] } } }, @@ -60489,14 +61396,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "integers_asset", - "solid_name": "integers_asset", + "solid_def_name": "return_integer", + "solid_name": "return_integer", "tags": {} } ] }, "description": null, - "graph_def_name": "integers_asset_job", + "graph_def_name": "integers", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -60531,43 +61438,14 @@ "name": "config", "type_key": "Any" }, - "description": null, - "name": "dummy_io_manager" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" - }, - "description": null, - "name": "hanging_asset_resource" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.370e26b257ba7c336c1b21a7bd8e340ea17d2bc2" + "root_config_key": "Shape.61678299d213abc6dacb4a403face86263e676d7" } ], - "name": "integers_asset_job", - "run_tags": {}, + "name": "integers", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -60585,7 +61463,7 @@ }, "description": null, "input_def_snaps": [], - "name": "integers_asset", + "name": "return_integer", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -60606,7 +61484,7 @@ ''' # --- # name: test_all_snapshot_ids[65] - '10e316cc85a348c2e9f5c5ec1a076bfe7e036ff2' + 'c4fee3a0c56b1ef6fcea9b64ceaacd06c4d5e216' # --- # name: test_all_snapshot_ids[66] ''' @@ -60769,6 +61647,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Selector.2571019f1a5201853d11032145ac3e534067f214": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "env", + "type_key": "String" + } + ], + "given_name": null, + "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -61052,30 +61953,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.2826887148df509057b8bdd850d72268525729fe": { + "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -61085,22 +61963,22 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "op_with_config", - "type_key": "Shape.3984261e8838dd039e5c0d108ad1c0ac6110ace1" + "is_required": false, + "name": "base_dir", + "type_key": "StringSourceType" } ], "given_name": null, - "key": "Shape.2826887148df509057b8bdd850d72268525729fe", + "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.37610aaf407f315481cab1fdecba2e7f238e0a62": { + "Shape.370e26b257ba7c336c1b21a7bd8e340ea17d2bc2": { "__class__": "ConfigTypeSnap", - "description": "This run config schema was automatically populated with default values from `default_config`.", + "description": null, "enum_values": null, "fields": [ { @@ -61124,47 +62002,47 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"op_with_config\": {\"config\": {\"one\": \"hullo\"}}}", + "default_value_as_json_str": "{\"integers_asset\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.2826887148df509057b8bdd850d72268525729fe" + "type_key": "Shape.431e85768ac5c1cb79e6f0893ca4a2e5dda4d4a3" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", + "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", "description": "Configure how shared resources are implemented within a run.", "is_required": false, "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" } ], "given_name": null, - "key": "Shape.37610aaf407f315481cab1fdecba2e7f238e0a62", + "key": "Shape.370e26b257ba7c336c1b21a7bd8e340ea17d2bc2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.3984261e8838dd039e5c0d108ad1c0ac6110ace1": { + "Shape.431e85768ac5c1cb79e6f0893ca4a2e5dda4d4a3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518" + "is_required": false, + "name": "integers_asset", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.3984261e8838dd039e5c0d108ad1c0ac6110ace1", + "key": "Shape.431e85768ac5c1cb79e6f0893ca4a2e5dda4d4a3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -61203,6 +62081,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + } + ], + "given_name": null, + "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, @@ -61249,7 +62150,48 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518": { + "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "dummy_io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {}}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + } + ], + "given_name": null, + "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -61260,12 +62202,35 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "one", + "name": "file", "type_key": "String" } ], "given_name": null, - "key": "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518", + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + } + ], + "given_name": null, + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -61322,6 +62287,22 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null + }, + "StringSourceType": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "StringSourceType", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "String", + "Selector.2571019f1a5201853d11032145ac3e534067f214" + ] } } }, @@ -61421,14 +62402,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_with_config", - "solid_name": "op_with_config", + "solid_def_name": "integers_asset", + "solid_name": "integers_asset", "tags": {} } ] }, "description": null, - "graph_def_name": "job_with_default_config", + "graph_def_name": "integers_asset_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -61463,14 +62444,43 @@ "name": "config", "type_key": "Any" }, + "description": null, + "name": "dummy_io_manager" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + }, + "description": null, + "name": "hanging_asset_resource" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.37610aaf407f315481cab1fdecba2e7f238e0a62" + "root_config_key": "Shape.370e26b257ba7c336c1b21a7bd8e340ea17d2bc2" } ], - "name": "job_with_default_config", + "name": "integers_asset_job", + "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -61482,13 +62492,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, + "is_required": false, "name": "config", - "type_key": "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518" + "type_key": "Any" }, "description": null, "input_def_snaps": [], - "name": "op_with_config", + "name": "integers_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -61509,7 +62519,7 @@ ''' # --- # name: test_all_snapshot_ids[67] - '643e2b02ca69b0087d15b448a9108d39d5b35036' + '10e316cc85a348c2e9f5c5ec1a076bfe7e036ff2' # --- # name: test_all_snapshot_ids[68] ''' @@ -61978,7 +62988,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.2826887148df509057b8bdd850d72268525729fe": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -61987,30 +62997,71 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "[DEPRECATED]", + "description": null, + "is_required": true, + "name": "op_with_config", + "type_key": "Shape.3984261e8838dd039e5c0d108ad1c0ac6110ace1" + } + ], + "given_name": null, + "key": "Shape.2826887148df509057b8bdd850d72268525729fe", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.37610aaf407f315481cab1fdecba2e7f238e0a62": { + "__class__": "ConfigTypeSnap", + "description": "This run config schema was automatically populated with default values from `default_config`.", + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "marker_to_close", - "type_key": "String" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"op_with_config\": {\"config\": {\"one\": \"hullo\"}}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.2826887148df509057b8bdd850d72268525729fe" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.37610aaf407f315481cab1fdecba2e7f238e0a62", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.3984261e8838dd039e5c0d108ad1c0ac6110ace1": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -62021,19 +63072,19 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "path", - "type_key": "String" + "name": "config", + "type_key": "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.3984261e8838dd039e5c0d108ad1c0ac6110ace1", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -62042,21 +63093,30 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, + "description": "[DEPRECATED]", "is_required": false, - "name": "config", - "type_key": "Any" + "name": "marker_to_close", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", + "is_required": false, + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.8ca73f0ae5a27b2d717a94a2cba479568ddc79b2": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -62067,19 +63127,19 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "config", - "type_key": "TestEnum" + "name": "path", + "type_key": "String" } ], "given_name": null, - "key": "Shape.8ca73f0ae5a27b2d717a94a2cba479568ddc79b2", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.b214dc99bad5dc89e19e327bf3c9abcd56c353db": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -62089,99 +63149,72 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "takes_an_enum", - "type_key": "Shape.8ca73f0ae5a27b2d717a94a2cba479568ddc79b2" + "is_required": false, + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.b214dc99bad5dc89e19e327bf3c9abcd56c353db", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { + "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, - "fields": [], + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "one", + "type_key": "String" + } + ], "given_name": null, - "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", + "key": "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { + "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "console", - "type_key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a" - } - ], + "fields": [], "given_name": null, - "key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b", + "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.fd83dcfe0c9126a9b5956827a350611e935e0643": { + "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "Configure runtime parameters for ops or assets.", - "is_required": true, - "name": "ops", - "type_key": "Shape.b214dc99bad5dc89e19e327bf3c9abcd56c353db" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "description": null, "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "console", + "type_key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a" } ], "given_name": null, - "key": "Shape.fd83dcfe0c9126a9b5956827a350611e935e0643", + "key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -62202,35 +63235,6 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null - }, - "TestEnum": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": [ - { - "__class__": "ConfigEnumValueSnap", - "description": "An enum value.", - "value": "ENUM_VALUE_ONE" - }, - { - "__class__": "ConfigEnumValueSnap", - "description": "An enum value.", - "value": "ENUM_VALUE_TWO" - }, - { - "__class__": "ConfigEnumValueSnap", - "description": "An enum value.", - "value": "ENUM_VALUE_THREE" - } - ], - "fields": null, - "given_name": "TestEnum", - "key": "TestEnum", - "kind": { - "__enum__": "ConfigTypeKind.ENUM" - }, - "scalar_kind": null, - "type_param_keys": null } } }, @@ -62330,14 +63334,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "takes_an_enum", - "solid_name": "takes_an_enum", + "solid_def_name": "op_with_config", + "solid_name": "op_with_config", "tags": {} } ] }, "description": null, - "graph_def_name": "job_with_enum_config", + "graph_def_name": "job_with_default_config", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -62376,10 +63380,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.fd83dcfe0c9126a9b5956827a350611e935e0643" + "root_config_key": "Shape.37610aaf407f315481cab1fdecba2e7f238e0a62" } ], - "name": "job_with_enum_config", + "name": "job_with_default_config", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -62393,11 +63397,11 @@ "description": null, "is_required": true, "name": "config", - "type_key": "TestEnum" + "type_key": "Shape.bc52b1c43d99d0004be108b5869dde9f4a119518" }, "description": null, "input_def_snaps": [], - "name": "takes_an_enum", + "name": "op_with_config", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -62418,7 +63422,7 @@ ''' # --- # name: test_all_snapshot_ids[69] - '158a79ab642707e63923c59c6abaf7960b36211e' + '643e2b02ca69b0087d15b448a9108d39d5b35036' # --- # name: test_all_snapshot_ids[6] ''' @@ -64024,6 +65028,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, @@ -64102,98 +65129,30 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.a775abaebbf7dbee4f702f3e27405e49639e5f85": { + "Shape.8ca73f0ae5a27b2d717a94a2cba479568ddc79b2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "emit_failed_expectation", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "emit_successful_expectation", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "emit_successful_expectation_no_metadata", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.a775abaebbf7dbee4f702f3e27405e49639e5f85", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.b0b6c15989be346c4b066371bcffeba8cff2fcb9": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"emit_failed_expectation\": {}, \"emit_successful_expectation\": {}, \"emit_successful_expectation_no_metadata\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.a775abaebbf7dbee4f702f3e27405e49639e5f85" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641" + "is_required": true, + "name": "config", + "type_key": "TestEnum" } ], "given_name": null, - "key": "Shape.b0b6c15989be346c4b066371bcffeba8cff2fcb9", + "key": "Shape.8ca73f0ae5a27b2d717a94a2cba479568ddc79b2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641": { + "Shape.b214dc99bad5dc89e19e327bf3c9abcd56c353db": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -64202,14 +65161,14 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "description": null, + "is_required": true, + "name": "takes_an_enum", + "type_key": "Shape.8ca73f0ae5a27b2d717a94a2cba479568ddc79b2" } ], "given_name": null, - "key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641", + "key": "Shape.b214dc99bad5dc89e19e327bf3c9abcd56c353db", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -64252,6 +65211,56 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.fd83dcfe0c9126a9b5956827a350611e935e0643": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Configure runtime parameters for ops or assets.", + "is_required": true, + "name": "ops", + "type_key": "Shape.b214dc99bad5dc89e19e327bf3c9abcd56c353db" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + } + ], + "given_name": null, + "key": "Shape.fd83dcfe0c9126a9b5956827a350611e935e0643", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -64266,6 +65275,35 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null + }, + "TestEnum": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": [ + { + "__class__": "ConfigEnumValueSnap", + "description": "An enum value.", + "value": "ENUM_VALUE_ONE" + }, + { + "__class__": "ConfigEnumValueSnap", + "description": "An enum value.", + "value": "ENUM_VALUE_TWO" + }, + { + "__class__": "ConfigEnumValueSnap", + "description": "An enum value.", + "value": "ENUM_VALUE_THREE" + } + ], + "fields": null, + "given_name": "TestEnum", + "key": "TestEnum", + "kind": { + "__enum__": "ConfigTypeKind.ENUM" + }, + "scalar_kind": null, + "type_param_keys": null } } }, @@ -64365,30 +65403,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "emit_failed_expectation", - "solid_name": "emit_failed_expectation", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "emit_successful_expectation", - "solid_name": "emit_successful_expectation", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "emit_successful_expectation_no_metadata", - "solid_name": "emit_successful_expectation_no_metadata", + "solid_def_name": "takes_an_enum", + "solid_name": "takes_an_enum", "tags": {} } ] }, "description": null, - "graph_def_name": "job_with_expectations", + "graph_def_name": "job_with_enum_config", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -64427,10 +65449,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.b0b6c15989be346c4b066371bcffeba8cff2fcb9" + "root_config_key": "Shape.fd83dcfe0c9126a9b5956827a350611e935e0643" } ], - "name": "job_with_expectations", + "name": "job_with_enum_config", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -64442,50 +65464,23 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "emit_failed_expectation", - "output_def_snaps": [], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "emit_successful_expectation", - "output_def_snaps": [], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "TestEnum" }, "description": null, "input_def_snaps": [], - "name": "emit_successful_expectation_no_metadata", - "output_def_snaps": [], + "name": "takes_an_enum", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], "required_resource_keys": [], "tags": {} } @@ -64496,7 +65491,7 @@ ''' # --- # name: test_all_snapshot_ids[71] - '25eed9832eef95ee97a63357e836659193775b3d' + '158a79ab642707e63923c59c6abaf7960b36211e' # --- # name: test_all_snapshot_ids[72] ''' @@ -64942,52 +65937,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.3c9fe9866f4fe467d426b7a9d3aeda21504ed154": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "op_with_input_output_metadata", - "type_key": "Shape.64a2e45f416adce23c20a991c4bfce13c2cbe2d5" - } - ], - "given_name": null, - "key": "Shape.3c9fe9866f4fe467d426b7a9d3aeda21504ed154", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, @@ -65043,7 +65992,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.64a2e45f416adce23c20a991c4bfce13c2cbe2d5": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -65056,26 +66005,58 @@ "is_required": false, "name": "config", "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.a775abaebbf7dbee4f702f3e27405e49639e5f85": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "emit_failed_expectation", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "inputs", - "type_key": "Shape.85b49263d1851d0d885cfec001faca6da5738880" + "is_required": false, + "name": "emit_successful_expectation", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "emit_successful_expectation_no_metadata", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.64a2e45f416adce23c20a991c4bfce13c2cbe2d5", + "key": "Shape.a775abaebbf7dbee4f702f3e27405e49639e5f85", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.69f3d02a3624c3bfbe54c903275932f1379dda2e": { + "Shape.b0b6c15989be346c4b066371bcffeba8cff2fcb9": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -65100,32 +66081,32 @@ }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{\"emit_failed_expectation\": {}, \"emit_successful_expectation\": {}, \"emit_successful_expectation_no_metadata\": {}}", "description": "Configure runtime parameters for ops or assets.", - "is_required": true, + "is_required": false, "name": "ops", - "type_key": "Shape.3c9fe9866f4fe467d426b7a9d3aeda21504ed154" + "type_key": "Shape.a775abaebbf7dbee4f702f3e27405e49639e5f85" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", + "default_value_as_json_str": "{}", "description": "Configure how shared resources are implemented within a run.", "is_required": false, "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "type_key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641" } ], "given_name": null, - "key": "Shape.69f3d02a3624c3bfbe54c903275932f1379dda2e", + "key": "Shape.b0b6c15989be346c4b066371bcffeba8cff2fcb9", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -65134,37 +66115,14 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "config", - "type_key": "Any" - } - ], - "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.85b49263d1851d0d885cfec001faca6da5738880": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "foo", - "type_key": "ScalarUnion.Int-Selector.a9799b971d12ace70a2d8803c883c863417d0725" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.85b49263d1851d0d885cfec001faca6da5738880", + "key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -65318,23 +66276,32 @@ "solid_invocation_snaps": [ { "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "foo", - "is_dynamic_collect": false, - "upstream_output_snaps": [] - } - ], + "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_with_input_output_metadata", - "solid_name": "op_with_input_output_metadata", + "solid_def_name": "emit_failed_expectation", + "solid_name": "emit_failed_expectation", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "emit_successful_expectation", + "solid_name": "emit_successful_expectation", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "emit_successful_expectation_no_metadata", + "solid_name": "emit_successful_expectation_no_metadata", "tags": {} } ] }, "description": null, - "graph_def_name": "job_with_input_output_metadata", + "graph_def_name": "job_with_expectations", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -65373,10 +66340,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.69f3d02a3624c3bfbe54c903275932f1379dda2e" + "root_config_key": "Shape.b0b6c15989be346c4b066371bcffeba8cff2fcb9" } ], - "name": "job_with_input_output_metadata", + "name": "job_with_expectations", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -65393,47 +66360,45 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "metadata_entries": [ - { - "__class__": "EventMetadataEntry", - "description": null, - "entry_data": { - "__class__": "TextMetadataEntryData", - "text": "b" - }, - "label": "a" - } - ], - "name": "foo" - } - ], - "name": "op_with_input_output_metadata", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Int", - "description": null, - "is_dynamic": false, - "is_required": true, - "metadata_entries": [ - { - "__class__": "EventMetadataEntry", - "description": null, - "entry_data": { - "__class__": "TextMetadataEntryData", - "text": "d" - }, - "label": "c" - } - ], - "name": "bar" - } - ], + "input_def_snaps": [], + "name": "emit_failed_expectation", + "output_def_snaps": [], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "emit_successful_expectation", + "output_def_snaps": [], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "emit_successful_expectation_no_metadata", + "output_def_snaps": [], "required_resource_keys": [], "tags": {} } @@ -65444,7 +66409,7 @@ ''' # --- # name: test_all_snapshot_ids[73] - '6069db2378a5bcd370c5f8ce8e8ee51d997e3447' + '25eed9832eef95ee97a63357e836659193775b3d' # --- # name: test_all_snapshot_ids[74] ''' @@ -65913,50 +66878,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.288595c17d5cb84370159ed6ea6d866b85f47f05": { + "Shape.3c9fe9866f4fe467d426b7a9d3aeda21504ed154": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"fail_subset\": {}, \"one\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.a14c2c695c8de0c8ebdad9323d40dd33323ef660" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "op_with_input_output_metadata", + "type_key": "Shape.64a2e45f416adce23c20a991c4bfce13c2cbe2d5" } ], "given_name": null, - "key": "Shape.288595c17d5cb84370159ed6ea6d866b85f47f05", + "key": "Shape.3c9fe9866f4fe467d426b7a9d3aeda21504ed154", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -66018,7 +66956,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.64a2e45f416adce23c20a991c4bfce13c2cbe2d5": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -66031,17 +66969,26 @@ "is_required": false, "name": "config", "type_key": "Any" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "inputs", + "type_key": "Shape.85b49263d1851d0d885cfec001faca6da5738880" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.64a2e45f416adce23c20a991c4bfce13c2cbe2d5", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.a14c2c695c8de0c8ebdad9323d40dd33323ef660": { + "Shape.69f3d02a3624c3bfbe54c903275932f1379dda2e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -66049,24 +66996,88 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "fail_subset", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Configure runtime parameters for ops or assets.", + "is_required": true, + "name": "ops", + "type_key": "Shape.3c9fe9866f4fe467d426b7a9d3aeda21504ed154" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + } + ], + "given_name": null, + "key": "Shape.69f3d02a3624c3bfbe54c903275932f1379dda2e", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "one", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.a14c2c695c8de0c8ebdad9323d40dd33323ef660", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.85b49263d1851d0d885cfec001faca6da5738880": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "foo", + "type_key": "ScalarUnion.Int-Selector.a9799b971d12ace70a2d8803c883c863417d0725" + } + ], + "given_name": null, + "key": "Shape.85b49263d1851d0d885cfec001faca6da5738880", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -66171,20 +67182,6 @@ "name": "Float", "type_param_keys": [] }, - "InputTypeWithoutHydration": { - "__class__": "DagsterTypeSnap", - "description": null, - "display_name": "InputTypeWithoutHydration", - "is_builtin": false, - "key": "InputTypeWithoutHydration", - "kind": { - "__enum__": "DagsterTypeKind.REGULAR" - }, - "loader_schema_key": null, - "materializer_schema_key": null, - "name": "InputTypeWithoutHydration", - "type_param_keys": [] - }, "Int": { "__class__": "DagsterTypeSnap", "description": null, @@ -66237,34 +67234,20 @@ "input_dep_snaps": [ { "__class__": "InputDependencySnap", - "input_name": "some_input", + "input_name": "foo", "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "one" - } - ] + "upstream_output_snaps": [] } ], "is_dynamic_mapped": false, - "solid_def_name": "fail_subset", - "solid_name": "fail_subset", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "one", - "solid_name": "one", + "solid_def_name": "op_with_input_output_metadata", + "solid_name": "op_with_input_output_metadata", "tags": {} } ] }, "description": null, - "graph_def_name": "job_with_invalid_definition_error", + "graph_def_name": "job_with_input_output_metadata", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -66303,10 +67286,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.288595c17d5cb84370159ed6ea6d866b85f47f05" + "root_config_key": "Shape.69f3d02a3624c3bfbe54c903275932f1379dda2e" } ], - "name": "job_with_invalid_definition_error", + "name": "job_with_input_output_metadata", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -66326,47 +67309,42 @@ "input_def_snaps": [ { "__class__": "InputDefSnap", - "dagster_type_key": "InputTypeWithoutHydration", - "description": null, - "name": "some_input" - } - ], - "name": "fail_subset", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", "dagster_type_key": "Int", "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" + "metadata_entries": [ + { + "__class__": "EventMetadataEntry", + "description": null, + "entry_data": { + "__class__": "TextMetadataEntryData", + "text": "b" + }, + "label": "a" + } + ], + "name": "foo" } ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "one", + "name": "op_with_input_output_metadata", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "InputTypeWithoutHydration", + "dagster_type_key": "Int", "description": null, "is_dynamic": false, "is_required": true, - "name": "result" + "metadata_entries": [ + { + "__class__": "EventMetadataEntry", + "description": null, + "entry_data": { + "__class__": "TextMetadataEntryData", + "text": "d" + }, + "label": "c" + } + ], + "name": "bar" } ], "required_resource_keys": [], @@ -66379,7 +67357,7 @@ ''' # --- # name: test_all_snapshot_ids[75] - 'de3cee0b30bf45c5c28b57035caff8a592cb8a99' + '6069db2378a5bcd370c5f8ce8e8ee51d997e3447' # --- # name: test_all_snapshot_ids[76] ''' @@ -66401,21 +67379,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Array.Int": { - "__class__": "ConfigTypeSnap", - "description": "List of Array.Int", - "enum_values": null, - "fields": null, - "given_name": null, - "key": "Array.Int", - "kind": { - "__enum__": "ConfigTypeKind.ARRAY" - }, - "scalar_kind": null, - "type_param_keys": [ - "Int" - ] - }, "Bool": { "__class__": "ConfigTypeSnap", "description": "", @@ -66840,30 +67803,30 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.12236b5e54169d18e86e0d47deacb0b076a14af8": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Array.Int" + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.12236b5e54169d18e86e0d47deacb0b076a14af8", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.147c68699311087d0426de0e96d2b6834556a246": { + "Shape.288595c17d5cb84370159ed6ea6d866b85f47f05": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -66888,48 +67851,25 @@ }, { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{\"fail_subset\": {}, \"one\": {}}", "description": "Configure runtime parameters for ops or assets.", - "is_required": true, + "is_required": false, "name": "ops", - "type_key": "Shape.3ff5846df24fb29b3480ee420e75c21b508bcf1d" + "type_key": "Shape.a14c2c695c8de0c8ebdad9323d40dd33323ef660" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", + "default_value_as_json_str": "{\"io_manager\": {}}", "description": "Configure how shared resources are implemented within a run.", "is_required": false, "name": "resources", - "type_key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641" - } - ], - "given_name": null, - "key": "Shape.147c68699311087d0426de0e96d2b6834556a246", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.3ff5846df24fb29b3480ee420e75c21b508bcf1d": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "op_with_list", - "type_key": "Shape.12236b5e54169d18e86e0d47deacb0b076a14af8" + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.3ff5846df24fb29b3480ee420e75c21b508bcf1d", + "key": "Shape.288595c17d5cb84370159ed6ea6d866b85f47f05", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -67014,23 +67954,32 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641": { + "Shape.a14c2c695c8de0c8ebdad9323d40dd33323ef660": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "io_manager", + "name": "fail_subset", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "one", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641", + "key": "Shape.a14c2c695c8de0c8ebdad9323d40dd33323ef660", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -67135,6 +68084,20 @@ "name": "Float", "type_param_keys": [] }, + "InputTypeWithoutHydration": { + "__class__": "DagsterTypeSnap", + "description": null, + "display_name": "InputTypeWithoutHydration", + "is_builtin": false, + "key": "InputTypeWithoutHydration", + "kind": { + "__enum__": "DagsterTypeKind.REGULAR" + }, + "loader_schema_key": null, + "materializer_schema_key": null, + "name": "InputTypeWithoutHydration", + "type_param_keys": [] + }, "Int": { "__class__": "DagsterTypeSnap", "description": null, @@ -67182,18 +68145,39 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "some_input", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "one" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "fail_subset", + "solid_name": "fail_subset", + "tags": {} + }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_with_list", - "solid_name": "op_with_list", + "solid_def_name": "one", + "solid_name": "one", "tags": {} } ] }, "description": null, - "graph_def_name": "job_with_list", + "graph_def_name": "job_with_invalid_definition_error", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -67232,10 +68216,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.147c68699311087d0426de0e96d2b6834556a246" + "root_config_key": "Shape.288595c17d5cb84370159ed6ea6d866b85f47f05" } ], - "name": "job_with_list", + "name": "job_with_invalid_definition_error", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -67247,14 +68231,57 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, + "is_required": false, "name": "config", - "type_key": "Array.Int" + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "InputTypeWithoutHydration", + "description": null, + "name": "some_input" + } + ], + "name": "fail_subset", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Int", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" }, "description": null, "input_def_snaps": [], - "name": "op_with_list", - "output_def_snaps": [], + "name": "one", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "InputTypeWithoutHydration", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], "required_resource_keys": [], "tags": {} } @@ -67265,7 +68292,7 @@ ''' # --- # name: test_all_snapshot_ids[77] - '68f90c6bc3483e01ab1317573d90e23d0efe14a9' + 'de3cee0b30bf45c5c28b57035caff8a592cb8a99' # --- # name: test_all_snapshot_ids[78] ''' @@ -67287,6 +68314,21 @@ "scalar_kind": null, "type_param_keys": null }, + "Array.Int": { + "__class__": "ConfigTypeSnap", + "description": "List of Array.Int", + "enum_values": null, + "fields": null, + "given_name": null, + "key": "Array.Int", + "kind": { + "__enum__": "ConfigTypeKind.ARRAY" + }, + "scalar_kind": null, + "type_param_keys": [ + "Int" + ] + }, "Bool": { "__class__": "ConfigTypeSnap", "description": "", @@ -67633,7 +68675,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.05f8c9463d76532175f0571ee1be34b39da6dc64": { + "Shape.081354663b9d4b8fbfd1cb8e358763912953913f": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -67641,42 +68683,24 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.bef10374a7619a637bcc228e2146e8ee88399f1f" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"return_six\": {}}", - "description": "Configure runtime parameters for ops or assets.", + "default_value_as_json_str": "\"INFO\"", + "description": "The logger's threshold.", "is_required": false, - "name": "ops", - "type_key": "Shape.1b7d26bbdb8188505a571fc3b5bba3348f1da7fa" + "name": "log_level", + "type_key": "String" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "default_value_as_json_str": "\"dagster\"", + "description": "The name of your logger.", "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "name", + "type_key": "String" } ], "given_name": null, - "key": "Shape.05f8c9463d76532175f0571ee1be34b39da6dc64", + "key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -67706,7 +68730,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -67714,70 +68738,111 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", + "description": "The default colored console logger.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.1b7d26bbdb8188505a571fc3b5bba3348f1da7fa": { + "Shape.12236b5e54169d18e86e0d47deacb0b076a14af8": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "return_six", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "is_required": true, + "name": "config", + "type_key": "Array.Int" } ], "given_name": null, - "key": "Shape.1b7d26bbdb8188505a571fc3b5bba3348f1da7fa", + "key": "Shape.12236b5e54169d18e86e0d47deacb0b076a14af8", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308": { + "Shape.147c68699311087d0426de0e96d2b6834556a246": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, + "description": "Configure runtime parameters for ops or assets.", "is_required": true, - "name": "log_level", - "type_key": "String" + "name": "ops", + "type_key": "Shape.3ff5846df24fb29b3480ee420e75c21b508bcf1d" }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641" + } + ], + "given_name": null, + "key": "Shape.147c68699311087d0426de0e96d2b6834556a246", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.3ff5846df24fb29b3480ee420e75c21b508bcf1d": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "prefix", - "type_key": "String" + "name": "op_with_list", + "type_key": "Shape.12236b5e54169d18e86e0d47deacb0b076a14af8" } ], "given_name": null, - "key": "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308", + "key": "Shape.3ff5846df24fb29b3480ee420e75c21b508bcf1d", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -67862,30 +68927,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.a793714b5918623c92ddb83973046ee8df58a423": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.a793714b5918623c92ddb83973046ee8df58a423", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.bef10374a7619a637bcc228e2146e8ee88399f1f": { + "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -67894,23 +68936,14 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "bar", - "type_key": "Shape.de97af1dab49255e847483b3b93bf53f75ab76d7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "foo", - "type_key": "Shape.a793714b5918623c92ddb83973046ee8df58a423" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.bef10374a7619a637bcc228e2146e8ee88399f1f", + "key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -67930,7 +68963,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.de97af1dab49255e847483b3b93bf53f75ab76d7": { + "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -67940,13 +68973,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308" + "is_required": false, + "name": "console", + "type_key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a" } ], "given_name": null, - "key": "Shape.de97af1dab49255e847483b3b93bf53f75ab76d7", + "key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -68066,14 +69099,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "return_six", - "solid_name": "return_six", + "solid_def_name": "op_with_list", + "solid_name": "op_with_list", "tags": {} } ] }, "description": null, - "graph_def_name": "loggers_job", + "graph_def_name": "job_with_list", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -68084,29 +69117,15 @@ "__class__": "LoggerDefSnap", "config_field_snap": { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308" - }, - "description": null, - "name": "bar" - }, - { - "__class__": "LoggerDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, + "default_provided": true, + "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", + "description": "The default colored console logger.", + "is_required": false, "name": "config", - "type_key": "String" + "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" }, - "description": null, - "name": "foo" + "description": "The default colored console logger.", + "name": "console" } ], "name": "default", @@ -68126,10 +69145,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.05f8c9463d76532175f0571ee1be34b39da6dc64" + "root_config_key": "Shape.147c68699311087d0426de0e96d2b6834556a246" } ], - "name": "loggers_job", + "name": "job_with_list", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -68141,23 +69160,14 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Array.Int" }, "description": null, "input_def_snaps": [], - "name": "return_six", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], + "name": "op_with_list", + "output_def_snaps": [], "required_resource_keys": [], "tags": {} } @@ -68168,7 +69178,7 @@ ''' # --- # name: test_all_snapshot_ids[79] - '1c4b82d393f7eacd9a358e3d82925d2afb9afbc0' + '68f90c6bc3483e01ab1317573d90e23d0efe14a9' # --- # name: test_all_snapshot_ids[7] '8023256507d183317343a2f6c4703a5ac3800eaf' @@ -68539,7 +69549,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.081354663b9d4b8fbfd1cb8e358763912953913f": { + "Shape.05f8c9463d76532175f0571ee1be34b39da6dc64": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -68547,24 +69557,42 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "\"INFO\"", - "description": "The logger's threshold.", + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "log_level", - "type_key": "String" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "\"dagster\"", - "description": "The name of your logger.", + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "name", - "type_key": "String" + "name": "loggers", + "type_key": "Shape.bef10374a7619a637bcc228e2146e8ee88399f1f" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"return_six\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.1b7d26bbdb8188505a571fc3b5bba3348f1da7fa" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f", + "key": "Shape.05f8c9463d76532175f0571ee1be34b39da6dc64", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -68594,7 +69622,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -68602,22 +69630,22 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", - "description": "The default colored console logger.", + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "config", - "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.1b7d26bbdb8188505a571fc3b5bba3348f1da7fa": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -68626,14 +69654,46 @@ "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "description": null, "is_required": false, - "name": "io_manager", + "name": "return_six", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.1b7d26bbdb8188505a571fc3b5bba3348f1da7fa", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "log_level", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "prefix", + "type_key": "String" + } + ], + "given_name": null, + "key": "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -68695,57 +69755,30 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.692ec318557c0a6cb437ddacf9c33c8aa0865045": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"materialize\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.84b306ab9af8f11de9541acce04f63760ecf48f5" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.692ec318557c0a6cb437ddacf9c33c8aa0865045", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.a793714b5918623c92ddb83973046ee8df58a423": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -68755,36 +69788,45 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "String" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.a793714b5918623c92ddb83973046ee8df58a423", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.84b306ab9af8f11de9541acce04f63760ecf48f5": { + "Shape.bef10374a7619a637bcc228e2146e8ee88399f1f": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "materialize", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "bar", + "type_key": "Shape.de97af1dab49255e847483b3b93bf53f75ab76d7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "foo", + "type_key": "Shape.a793714b5918623c92ddb83973046ee8df58a423" } ], "given_name": null, - "key": "Shape.84b306ab9af8f11de9541acce04f63760ecf48f5", + "key": "Shape.bef10374a7619a637bcc228e2146e8ee88399f1f", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -68804,7 +69846,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { + "Shape.de97af1dab49255e847483b3b93bf53f75ab76d7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -68814,13 +69856,13 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "console", - "type_key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a" + "is_required": true, + "name": "config", + "type_key": "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308" } ], "given_name": null, - "key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b", + "key": "Shape.de97af1dab49255e847483b3b93bf53f75ab76d7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -68940,14 +69982,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "materialize", - "solid_name": "materialize", + "solid_def_name": "return_six", + "solid_name": "return_six", "tags": {} } ] }, "description": null, - "graph_def_name": "materialization_job", + "graph_def_name": "loggers_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -68958,15 +70000,29 @@ "__class__": "LoggerDefSnap", "config_field_snap": { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", - "description": "The default colored console logger.", - "is_required": false, + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, "name": "config", - "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" + "type_key": "Shape.37a85e53844c27abff7868c5c2e6a32d7fd1c308" }, - "description": "The default colored console logger.", - "name": "console" + "description": null, + "name": "bar" + }, + { + "__class__": "LoggerDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "String" + }, + "description": null, + "name": "foo" } ], "name": "default", @@ -68986,10 +70042,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.692ec318557c0a6cb437ddacf9c33c8aa0865045" + "root_config_key": "Shape.05f8c9463d76532175f0571ee1be34b39da6dc64" } ], - "name": "materialization_job", + "name": "loggers_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -69007,7 +70063,7 @@ }, "description": null, "input_def_snaps": [], - "name": "materialize", + "name": "return_six", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -69028,7 +70084,7 @@ ''' # --- # name: test_all_snapshot_ids[81] - 'ab213b2b0286d659e9d7044d7dcec9b13d5b8bc7' + '1c4b82d393f7eacd9a358e3d82925d2afb9afbc0' # --- # name: test_all_snapshot_ids[82] ''' @@ -69497,47 +70553,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "field_one", - "type_key": "String" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "\"some_value\"", - "description": null, - "is_required": false, - "name": "field_three", - "type_key": "String" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "field_two", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, @@ -69593,78 +70608,96 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.692ec318557c0a6cb437ddacf9c33c8aa0865045": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "config", - "type_key": "Any" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"materialize\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.84b306ab9af8f11de9541acce04f63760ecf48f5" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.692ec318557c0a6cb437ddacf9c33c8aa0865045", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.c9dbd27648a17b3d172e0f5d1a2fdb30cc6fbc20": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "noop_op", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "op_with_three_field_config", - "type_key": "Shape.d2f148dd825e3845ea07d4ee0711dfe805d109a0" + "is_required": false, + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.c9dbd27648a17b3d172e0f5d1a2fdb30cc6fbc20", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.d2f148dd825e3845ea07d4ee0711dfe805d109a0": { + "Shape.84b306ab9af8f11de9541acce04f63760ecf48f5": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef" + "is_required": false, + "name": "materialize", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.d2f148dd825e3845ea07d4ee0711dfe805d109a0", + "key": "Shape.84b306ab9af8f11de9541acce04f63760ecf48f5", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -69707,56 +70740,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.f0b6efadf5d97e5f8d6c2ac1cc684b036af6b0fb": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Configure runtime parameters for ops or assets.", - "is_required": true, - "name": "ops", - "type_key": "Shape.c9dbd27648a17b3d172e0f5d1a2fdb30cc6fbc20" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" - } - ], - "given_name": null, - "key": "Shape.f0b6efadf5d97e5f8d6c2ac1cc684b036af6b0fb", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -69870,22 +70853,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "noop_op", - "solid_name": "noop_op", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "op_with_three_field_config", - "solid_name": "op_with_three_field_config", + "solid_def_name": "materialize", + "solid_name": "materialize", "tags": {} } ] }, "description": null, - "graph_def_name": "more_complicated_config", + "graph_def_name": "materialization_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -69924,10 +70899,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.f0b6efadf5d97e5f8d6c2ac1cc684b036af6b0fb" + "root_config_key": "Shape.692ec318557c0a6cb437ddacf9c33c8aa0865045" } ], - "name": "more_complicated_config", + "name": "materialization_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -69945,34 +70920,7 @@ }, "description": null, "input_def_snaps": [], - "name": "noop_op", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef" - }, - "description": null, - "input_def_snaps": [], - "name": "op_with_three_field_config", + "name": "materialize", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -69993,7 +70941,7 @@ ''' # --- # name: test_all_snapshot_ids[83] - '06068d3cd0c89f54330c52e56f4e72a338f19a9f' + 'ab213b2b0286d659e9d7044d7dcec9b13d5b8bc7' # --- # name: test_all_snapshot_ids[84] ''' @@ -70015,21 +70963,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Array.Noneable.Int": { - "__class__": "ConfigTypeSnap", - "description": "List of Array.Noneable.Int", - "enum_values": null, - "fields": null, - "given_name": null, - "key": "Array.Noneable.Int", - "kind": { - "__enum__": "ConfigTypeKind.ARRAY" - }, - "scalar_kind": null, - "type_param_keys": [ - "Noneable.Int" - ] - }, "Bool": { "__class__": "ConfigTypeSnap", "description": "", @@ -70075,21 +71008,6 @@ }, "type_param_keys": null }, - "Noneable.Int": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": null, - "given_name": null, - "key": "Noneable.Int", - "kind": { - "__enum__": "ConfigTypeKind.NONEABLE" - }, - "scalar_kind": null, - "type_param_keys": [ - "Int" - ] - }, "ScalarUnion.Bool-Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59": { "__class__": "ConfigTypeSnap", "description": null, @@ -70446,7 +71364,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.0c7ee2b6524bbf1be88f5b39b35e2c058d1be8ca": { + "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -70454,49 +71372,22 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Configure runtime parameters for ops or assets.", - "is_required": true, - "name": "ops", - "type_key": "Shape.f1d394720f3f1aa603eea869d2631d61d9a9751a" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how shared resources are implemented within a run.", + "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", + "description": "The default colored console logger.", "is_required": false, - "name": "resources", - "type_key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641" + "name": "config", + "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" } ], "given_name": null, - "key": "Shape.0c7ee2b6524bbf1be88f5b39b35e2c058d1be8ca", + "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -70504,22 +71395,22 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", - "description": "The default colored console logger.", + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "config", - "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -70528,30 +71419,39 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "[DEPRECATED]", - "is_required": false, - "name": "marker_to_close", + "description": null, + "is_required": true, + "name": "field_one", "type_key": "String" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", + "default_value_as_json_str": "\"some_value\"", + "description": null, "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + "name": "field_three", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "field_two", + "type_key": "String" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -70560,21 +71460,30 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "path", + "description": "[DEPRECATED]", + "is_required": false, + "name": "marker_to_close", "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", + "is_required": false, + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.54be9e9c03f5954ca8d86390f7b2b834f491973c": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -70585,30 +71494,12 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "field_five_int", - "type_key": "Int" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "field_four_str", + "name": "path", "type_key": "String" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "field_six_nullable_int_list", - "type_key": "Array.Noneable.Int" } ], "given_name": null, - "key": "Shape.54be9e9c03f5954ca8d86390f7b2b834f491973c", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -70638,46 +71529,19 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1": { + "Shape.c9dbd27648a17b3d172e0f5d1a2fdb30cc6fbc20": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "field_any", - "type_key": "Any" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "field_one", - "type_key": "String" - }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "\"some_value\"", - "description": null, - "is_required": false, - "name": "field_three", - "type_key": "String" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "field_two", - "type_key": "String" + "name": "noop_op", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", @@ -70685,19 +71549,19 @@ "default_value_as_json_str": null, "description": null, "is_required": true, - "name": "nested_field", - "type_key": "Shape.54be9e9c03f5954ca8d86390f7b2b834f491973c" + "name": "op_with_three_field_config", + "type_key": "Shape.d2f148dd825e3845ea07d4ee0711dfe805d109a0" } ], "given_name": null, - "key": "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1", + "key": "Shape.c9dbd27648a17b3d172e0f5d1a2fdb30cc6fbc20", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.9b6a380024ac2fa93165a958d31c9ca05450f68c": { + "Shape.d2f148dd825e3845ea07d4ee0711dfe805d109a0": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -70709,34 +71573,11 @@ "description": null, "is_required": true, "name": "config", - "type_key": "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1" - } - ], - "given_name": null, - "key": "Shape.9b6a380024ac2fa93165a958d31c9ca05450f68c", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "type_key": "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef" } ], "given_name": null, - "key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641", + "key": "Shape.d2f148dd825e3845ea07d4ee0711dfe805d109a0", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -70779,23 +71620,50 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.f1d394720f3f1aa603eea869d2631d61d9a9751a": { + "Shape.f0b6efadf5d97e5f8d6c2ac1cc684b036af6b0fb": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, { "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": null, + "description": "Configure runtime parameters for ops or assets.", "is_required": true, - "name": "op_with_multilayered_config", - "type_key": "Shape.9b6a380024ac2fa93165a958d31c9ca05450f68c" + "name": "ops", + "type_key": "Shape.c9dbd27648a17b3d172e0f5d1a2fdb30cc6fbc20" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.f1d394720f3f1aa603eea869d2631d61d9a9751a", + "key": "Shape.f0b6efadf5d97e5f8d6c2ac1cc684b036af6b0fb", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -70915,14 +71783,22 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_with_multilayered_config", - "solid_name": "op_with_multilayered_config", + "solid_def_name": "noop_op", + "solid_name": "noop_op", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "op_with_three_field_config", + "solid_name": "op_with_three_field_config", "tags": {} } ] }, "description": null, - "graph_def_name": "more_complicated_nested_config", + "graph_def_name": "more_complicated_config", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -70961,14 +71837,41 @@ "name": "io_manager" } ], - "root_config_key": "Shape.0c7ee2b6524bbf1be88f5b39b35e2c058d1be8ca" + "root_config_key": "Shape.f0b6efadf5d97e5f8d6c2ac1cc684b036af6b0fb" } ], - "name": "more_complicated_nested_config", + "name": "more_complicated_config", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], "solid_def_snaps": [ + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "noop_op", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -70978,12 +71881,21 @@ "description": null, "is_required": true, "name": "config", - "type_key": "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1" + "type_key": "Shape.229b6731a336bf9c6372da5f99a6c27bc6d086ef" }, "description": null, "input_def_snaps": [], - "name": "op_with_multilayered_config", - "output_def_snaps": [], + "name": "op_with_three_field_config", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], "required_resource_keys": [], "tags": {} } @@ -70994,7 +71906,7 @@ ''' # --- # name: test_all_snapshot_ids[85] - '10cfd966244a4e4064ff00517d3e012ad6ce4c7d' + '06068d3cd0c89f54330c52e56f4e72a338f19a9f' # --- # name: test_all_snapshot_ids[86] ''' @@ -71016,6 +71928,21 @@ "scalar_kind": null, "type_param_keys": null }, + "Array.Noneable.Int": { + "__class__": "ConfigTypeSnap", + "description": "List of Array.Noneable.Int", + "enum_values": null, + "fields": null, + "given_name": null, + "key": "Array.Noneable.Int", + "kind": { + "__enum__": "ConfigTypeKind.ARRAY" + }, + "scalar_kind": null, + "type_param_keys": [ + "Noneable.Int" + ] + }, "Bool": { "__class__": "ConfigTypeSnap", "description": "", @@ -71061,6 +71988,21 @@ }, "type_param_keys": null }, + "Noneable.Int": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "Noneable.Int", + "kind": { + "__enum__": "ConfigTypeKind.NONEABLE" + }, + "scalar_kind": null, + "type_param_keys": [ + "Int" + ] + }, "ScalarUnion.Bool-Selector.be5d518b39e86a43c5f2eecaf538c1f6c7711b59": { "__class__": "ConfigTypeSnap", "description": null, @@ -71417,7 +72359,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { + "Shape.0c7ee2b6524bbf1be88f5b39b35e2c058d1be8ca": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -71425,22 +72367,49 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", - "description": "The default colored console logger.", + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "config", - "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Configure runtime parameters for ops or assets.", + "is_required": true, + "name": "ops", + "type_key": "Shape.f1d394720f3f1aa603eea869d2631d61d9a9751a" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641" } ], "given_name": null, - "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", + "key": "Shape.0c7ee2b6524bbf1be88f5b39b35e2c058d1be8ca", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -71448,15 +72417,15 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_value_as_json_str": "{\"log_level\": \"INFO\", \"name\": \"dagster\"}", + "description": "The default colored console logger.", "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "config", + "type_key": "Shape.081354663b9d4b8fbfd1cb8e358763912953913f" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.0fe8353d6b542accfad9becbdbaeb92f649ebb9a", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -71518,89 +72487,130 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.4d6705b295892273089d4134b6d270debdb03ce3": { + "Shape.54be9e9c03f5954ca8d86390f7b2b834f491973c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, - "is_required": false, - "name": "op_asset_a", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "is_required": true, + "name": "field_five_int", + "type_key": "Int" }, { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "field_four_str", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "op_asset_b", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "field_six_nullable_int_list", + "type_key": "Array.Noneable.Int" } ], "given_name": null, - "key": "Shape.4d6705b295892273089d4134b6d270debdb03ce3", + "key": "Shape.54be9e9c03f5954ca8d86390f7b2b834f491973c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.68420c55741a33b18ef71c1b56d89e0ec12d5c06": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + "name": "config", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "field_any", + "type_key": "Any" }, { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "field_one", + "type_key": "String" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"op_asset_a\": {}, \"op_asset_b\": {}}", - "description": "Configure runtime parameters for ops or assets.", + "default_value_as_json_str": "\"some_value\"", + "description": null, "is_required": false, - "name": "ops", - "type_key": "Shape.4d6705b295892273089d4134b6d270debdb03ce3" + "name": "field_three", + "type_key": "String" }, { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "field_two", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "nested_field", + "type_key": "Shape.54be9e9c03f5954ca8d86390f7b2b834f491973c" } ], "given_name": null, - "key": "Shape.68420c55741a33b18ef71c1b56d89e0ec12d5c06", + "key": "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.9b6a380024ac2fa93165a958d31c9ca05450f68c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -71610,13 +72620,36 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.9b6a380024ac2fa93165a958d31c9ca05450f68c", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.d5cca4fabdc075ea3ec659dbeaffef0304d66641", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -71659,6 +72692,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.f1d394720f3f1aa603eea869d2631d61d9a9751a": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "op_with_multilayered_config", + "type_key": "Shape.9b6a380024ac2fa93165a958d31c9ca05450f68c" + } + ], + "given_name": null, + "key": "Shape.f1d394720f3f1aa603eea869d2631d61d9a9751a", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "String": { "__class__": "ConfigTypeSnap", "description": "", @@ -71772,35 +72828,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "op_asset_a", - "solid_name": "op_asset_a", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "num", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "op_asset_a" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "op_asset_b", - "solid_name": "op_asset_b", + "solid_def_name": "op_with_multilayered_config", + "solid_name": "op_with_multilayered_config", "tags": {} } ] }, "description": null, - "graph_def_name": "multi_asset_job", + "graph_def_name": "more_complicated_nested_config", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -71839,10 +72874,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.68420c55741a33b18ef71c1b56d89e0ec12d5c06" + "root_config_key": "Shape.0c7ee2b6524bbf1be88f5b39b35e2c058d1be8ca" } ], - "name": "multi_asset_job", + "name": "more_complicated_nested_config", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -71854,57 +72889,14 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": false, + "is_required": true, "name": "config", - "type_key": "Any" + "type_key": "Shape.7f086e66d4831dbcc8fea7cc9fe137292bad2bd1" }, "description": null, "input_def_snaps": [], - "name": "op_asset_a", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "num" - } - ], - "name": "op_asset_b", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], + "name": "op_with_multilayered_config", + "output_def_snaps": [], "required_resource_keys": [], "tags": {} } @@ -71915,7 +72907,7 @@ ''' # --- # name: test_all_snapshot_ids[87] - 'b7499dc1851fe912eb84bbbcead7d7db511e64d8' + '10cfd966244a4e4064ff00517d3e012ad6ce4c7d' # --- # name: test_all_snapshot_ids[88] ''' @@ -72078,29 +73070,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Selector.2571019f1a5201853d11032145ac3e534067f214": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "env", - "type_key": "String" - } - ], - "given_name": null, - "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", - "kind": { - "__enum__": "ConfigTypeKind.SELECTOR" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -72384,23 +73353,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { + "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "base_dir", - "type_key": "StringSourceType" + "name": "io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", + "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -72439,52 +73408,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - } - ], - "given_name": null, - "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.452c92bbdda7f588565ec0173b20a839231a469e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "multipartitions_fail", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.452c92bbdda7f588565ec0173b20a839231a469e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, @@ -72508,117 +73431,39 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - } - ], - "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "Shape.4d6705b295892273089d4134b6d270debdb03ce3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "dummy_io_manager", + "name": "op_asset_a", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" - }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" - } - ], - "given_name": null, - "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "file", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + "is_required": false, + "name": "op_asset_b", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", + "key": "Shape.4d6705b295892273089d4134b6d270debdb03ce3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.cc4977fdf89f5a2fa38ff7a5d4c3afd6c11a43a2": { + "Shape.68420c55741a33b18ef71c1b56d89e0ec12d5c06": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -72644,24 +73489,47 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"multipartitions_fail\": {}}", + "default_value_as_json_str": "{\"op_asset_a\": {}, \"op_asset_b\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.452c92bbdda7f588565ec0173b20a839231a469e" + "type_key": "Shape.4d6705b295892273089d4134b6d270debdb03ce3" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", + "default_value_as_json_str": "{\"io_manager\": {}}", "description": "Configure how shared resources are implemented within a run.", "is_required": false, "name": "resources", - "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.cc4977fdf89f5a2fa38ff7a5d4c3afd6c11a43a2", + "key": "Shape.68420c55741a33b18ef71c1b56d89e0ec12d5c06", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -72718,22 +73586,6 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null - }, - "StringSourceType": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": null, - "given_name": null, - "key": "StringSourceType", - "kind": { - "__enum__": "ConfigTypeKind.SCALAR_UNION" - }, - "scalar_kind": null, - "type_param_keys": [ - "String", - "Selector.2571019f1a5201853d11032145ac3e534067f214" - ] } } }, @@ -72833,14 +73685,35 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "multipartitions_fail", - "solid_name": "multipartitions_fail", + "solid_def_name": "op_asset_a", + "solid_name": "op_asset_a", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "num", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "op_asset_a" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "op_asset_b", + "solid_name": "op_asset_b", "tags": {} } ] }, "description": null, - "graph_def_name": "multipartitions_fail_job", + "graph_def_name": "multi_asset_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -72875,43 +73748,14 @@ "name": "config", "type_key": "Any" }, - "description": null, - "name": "dummy_io_manager" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" - }, - "description": null, - "name": "hanging_asset_resource" - }, - { - "__class__": "ResourceDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.cc4977fdf89f5a2fa38ff7a5d4c3afd6c11a43a2" + "root_config_key": "Shape.68420c55741a33b18ef71c1b56d89e0ec12d5c06" } ], - "name": "multipartitions_fail_job", - "run_tags": {}, + "name": "multi_asset_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -72929,7 +73773,41 @@ }, "description": null, "input_def_snaps": [], - "name": "multipartitions_fail", + "name": "op_asset_a", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "num" + } + ], + "name": "op_asset_b", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -72950,7 +73828,7 @@ ''' # --- # name: test_all_snapshot_ids[89] - '17bea22424bd5aa8995cc8ae08cd2e0c1be7c019' + 'b7499dc1851fe912eb84bbbcead7d7db511e64d8' # --- # name: test_all_snapshot_ids[8] ''' @@ -74351,73 +75229,46 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.452c92bbdda7f588565ec0173b20a839231a469e": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, - "is_required": true, - "name": "path", - "type_key": "String" + "is_required": false, + "name": "multipartitions_fail", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.452c92bbdda7f588565ec0173b20a839231a469e", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.67324aa04111ddc02877951d50eb6030ebbb1380": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"multipartitions_1\": {}, \"multipartitions_2\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.e5ada7fdb5d104210dab388858ab047e8cee0f41" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "path", + "type_key": "String" } ], "given_name": null, - "key": "Shape.67324aa04111ddc02877951d50eb6030ebbb1380", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -74534,20 +75385,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [], - "given_name": null, - "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.e5ada7fdb5d104210dab388858ab047e8cee0f41": { + "Shape.cc4977fdf89f5a2fa38ff7a5d4c3afd6c11a43a2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -74555,24 +75393,55 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "multipartitions_1", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "multipartitions_2", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"multipartitions_fail\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.452c92bbdda7f588565ec0173b20a839231a469e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" } ], "given_name": null, - "key": "Shape.e5ada7fdb5d104210dab388858ab047e8cee0f41", + "key": "Shape.cc4977fdf89f5a2fa38ff7a5d4c3afd6c11a43a2", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [], + "given_name": null, + "key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -74731,35 +75600,14 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "multipartitions_1", - "solid_name": "multipartitions_1", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "multipartitions_1", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "multipartitions_1" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "multipartitions_2", - "solid_name": "multipartitions_2", + "solid_def_name": "multipartitions_fail", + "solid_name": "multipartitions_fail", "tags": {} } ] }, "description": null, - "graph_def_name": "multipartitions_job", + "graph_def_name": "multipartitions_fail_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -74826,10 +75674,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.67324aa04111ddc02877951d50eb6030ebbb1380" + "root_config_key": "Shape.cc4977fdf89f5a2fa38ff7a5d4c3afd6c11a43a2" } ], - "name": "multipartitions_job", + "name": "multipartitions_fail_job", "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", @@ -74848,41 +75696,7 @@ }, "description": null, "input_def_snaps": [], - "name": "multipartitions_1", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "multipartitions_1" - } - ], - "name": "multipartitions_2", + "name": "multipartitions_fail", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -74903,7 +75717,7 @@ ''' # --- # name: test_all_snapshot_ids[91] - '7fc53a5a9f9ab514b23b938f7fa68cd5fd2f464d' + '17bea22424bd5aa8995cc8ae08cd2e0c1be7c019' # --- # name: test_all_snapshot_ids[92] ''' @@ -75473,30 +76287,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - } - ], - "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.7c17e21345d78710103d43ff7b2b039a1e23b8b3": { + "Shape.67324aa04111ddc02877951d50eb6030ebbb1380": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -75522,11 +76313,11 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"grouped_asset_1\": {}, \"grouped_asset_2\": {}, \"grouped_asset_4\": {}, \"ungrouped_asset_3\": {}, \"ungrouped_asset_5\": {}}", + "default_value_as_json_str": "{\"multipartitions_1\": {}, \"multipartitions_2\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.96e9540c3093065c72c5ac81d9efa07ba1affbd7" + "type_key": "Shape.e5ada7fdb5d104210dab388858ab047e8cee0f41" }, { "__class__": "ConfigFieldSnap", @@ -75539,14 +76330,14 @@ } ], "given_name": null, - "key": "Shape.7c17e21345d78710103d43ff7b2b039a1e23b8b3", + "key": "Shape.67324aa04111ddc02877951d50eb6030ebbb1380", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -75557,89 +76348,53 @@ "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "dummy_io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.96e9540c3093065c72c5ac81d9efa07ba1affbd7": { + "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "grouped_asset_1", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "grouped_asset_2", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "grouped_asset_4", + "name": "dummy_io_manager", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, - "name": "ungrouped_asset_3", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"config\": {}}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "is_required": false, - "name": "ungrouped_asset_5", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "io_manager", + "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" } ], "given_name": null, - "key": "Shape.96e9540c3093065c72c5ac81d9efa07ba1affbd7", + "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -75705,6 +76460,38 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.e5ada7fdb5d104210dab388858ab047e8cee0f41": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "multipartitions_1", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "multipartitions_2", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.e5ada7fdb5d104210dab388858ab047e8cee0f41", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b": { "__class__": "ConfigTypeSnap", "description": null, @@ -75857,46 +76644,35 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "grouped_asset_1", - "solid_name": "grouped_asset_1", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "grouped_asset_2", - "solid_name": "grouped_asset_2", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "grouped_asset_4", - "solid_name": "grouped_asset_4", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "ungrouped_asset_3", - "solid_name": "ungrouped_asset_3", + "solid_def_name": "multipartitions_1", + "solid_name": "multipartitions_1", "tags": {} }, { "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "multipartitions_1", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "multipartitions_1" + } + ] + } + ], "is_dynamic_mapped": false, - "solid_def_name": "ungrouped_asset_5", - "solid_name": "ungrouped_asset_5", + "solid_def_name": "multipartitions_2", + "solid_name": "multipartitions_2", "tags": {} } ] }, "description": null, - "graph_def_name": "named_groups_job", + "graph_def_name": "multipartitions_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -75963,10 +76739,10 @@ "name": "io_manager" } ], - "root_config_key": "Shape.7c17e21345d78710103d43ff7b2b039a1e23b8b3" + "root_config_key": "Shape.67324aa04111ddc02877951d50eb6030ebbb1380" } ], - "name": "named_groups_job", + "name": "multipartitions_job", "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", @@ -75985,61 +76761,7 @@ }, "description": null, "input_def_snaps": [], - "name": "grouped_asset_1", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "grouped_asset_2", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "grouped_asset_4", + "name": "multipartitions_1", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -76065,35 +76787,15 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "ungrouped_asset_3", - "output_def_snaps": [ + "input_def_snaps": [ { - "__class__": "OutputDefSnap", + "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" + "name": "multipartitions_1" } ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "ungrouped_asset_5", + "name": "multipartitions_2", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -76114,7 +76816,7 @@ ''' # --- # name: test_all_snapshot_ids[93] - 'f08d1a06b69649912ba8b056d76b8c7809954c31' + '7fc53a5a9f9ab514b23b938f7fa68cd5fd2f464d' # --- # name: test_all_snapshot_ids[94] ''' @@ -76277,6 +76979,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Selector.2571019f1a5201853d11032145ac3e534067f214": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "env", + "type_key": "String" + } + ], + "given_name": null, + "key": "Selector.2571019f1a5201853d11032145ac3e534067f214", + "kind": { + "__enum__": "ConfigTypeKind.SELECTOR" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Selector.a9799b971d12ace70a2d8803c883c863417d0725": { "__class__": "ConfigTypeSnap", "description": null, @@ -76560,23 +77285,23 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778": { + "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, "is_required": false, - "name": "io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "base_dir", + "type_key": "StringSourceType" } ], "given_name": null, - "key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778", + "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -76615,6 +77340,29 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + } + ], + "given_name": null, + "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, @@ -76661,7 +77409,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.9921ba4c33a841f7499fd90a37710a02cf33b795": { + "Shape.7c17e21345d78710103d43ff7b2b039a1e23b8b3": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -76687,31 +77435,72 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"throw_a_thing\": {}}", + "default_value_as_json_str": "{\"grouped_asset_1\": {}, \"grouped_asset_2\": {}, \"grouped_asset_4\": {}, \"ungrouped_asset_3\": {}, \"ungrouped_asset_5\": {}}", "description": "Configure runtime parameters for ops or assets.", "is_required": false, "name": "ops", - "type_key": "Shape.b3443ab7f0f2bae987a6d21cad9cfe00784f6ccb" + "type_key": "Shape.96e9540c3093065c72c5ac81d9efa07ba1affbd7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", + "default_value_as_json_str": "{\"io_manager\": {\"config\": {}}}", "description": "Configure how shared resources are implemented within a run.", "is_required": false, "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "type_key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d" } ], "given_name": null, - "key": "Shape.9921ba4c33a841f7499fd90a37710a02cf33b795", + "key": "Shape.7c17e21345d78710103d43ff7b2b039a1e23b8b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.b3443ab7f0f2bae987a6d21cad9cfe00784f6ccb": { + "Shape.941705f8db419acef3ba14bcecbc37e693570d4d": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "dummy_io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {}}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + } + ], + "given_name": null, + "key": "Shape.941705f8db419acef3ba14bcecbc37e693570d4d", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.96e9540c3093065c72c5ac81d9efa07ba1affbd7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -76722,12 +77511,94 @@ "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "throw_a_thing", + "name": "grouped_asset_1", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "grouped_asset_2", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "grouped_asset_4", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "ungrouped_asset_3", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "ungrouped_asset_5", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.b3443ab7f0f2bae987a6d21cad9cfe00784f6ccb", + "key": "Shape.96e9540c3093065c72c5ac81d9efa07ba1affbd7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "file", + "type_key": "String" + } + ], + "given_name": null, + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + } + ], + "given_name": null, + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -76784,6 +77655,22 @@ "__enum__": "ConfigScalarKind.STRING" }, "type_param_keys": null + }, + "StringSourceType": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": null, + "given_name": null, + "key": "StringSourceType", + "kind": { + "__enum__": "ConfigTypeKind.SCALAR_UNION" + }, + "scalar_kind": null, + "type_param_keys": [ + "String", + "Selector.2571019f1a5201853d11032145ac3e534067f214" + ] } } }, @@ -76883,14 +77770,46 @@ "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "throw_a_thing", - "solid_name": "throw_a_thing", + "solid_def_name": "grouped_asset_1", + "solid_name": "grouped_asset_1", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "grouped_asset_2", + "solid_name": "grouped_asset_2", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "grouped_asset_4", + "solid_name": "grouped_asset_4", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "ungrouped_asset_3", + "solid_name": "ungrouped_asset_3", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "ungrouped_asset_5", + "solid_name": "ungrouped_asset_5", "tags": {} } ] }, "description": null, - "graph_def_name": "naughty_programmer_job", + "graph_def_name": "named_groups_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -76925,14 +77844,43 @@ "name": "config", "type_key": "Any" }, + "description": null, + "name": "dummy_io_manager" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + }, + "description": null, + "name": "hanging_asset_resource" + }, + { + "__class__": "ResourceDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + }, "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", "name": "io_manager" } ], - "root_config_key": "Shape.9921ba4c33a841f7499fd90a37710a02cf33b795" + "root_config_key": "Shape.7c17e21345d78710103d43ff7b2b039a1e23b8b3" } ], - "name": "naughty_programmer_job", + "name": "named_groups_job", + "run_tags": {}, "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", "composite_solid_def_snaps": [], @@ -76950,7 +77898,115 @@ }, "description": null, "input_def_snaps": [], - "name": "throw_a_thing", + "name": "grouped_asset_1", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "grouped_asset_2", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "grouped_asset_4", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "ungrouped_asset_3", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "ungrouped_asset_5", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -76971,7 +78027,7 @@ ''' # --- # name: test_all_snapshot_ids[95] - '913c310b609478d52a81ee83bdd4b095d0f2932d' + 'f08d1a06b69649912ba8b056d76b8c7809954c31' # --- # name: test_all_snapshot_ids[96] ''' @@ -77440,112 +78496,39 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.27d4be3152e89c65ecb4ce8c588d8226cc827e0d": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"adder\": {}, \"op_1\": {}, \"op_2\": {}, \"plus_one\": {}}", - "description": null, - "is_required": false, - "name": "ops", - "type_key": "Shape.5403fe5383515d106ffe3bb7f1a927b1cbb4e8a9" - } - ], - "given_name": null, - "key": "Shape.27d4be3152e89c65ecb4ce8c588d8226cc827e0d", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.360ea318ffe78a111434a5bb7409ef66c9692290": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"plus_one\": {}, \"subgraph\": {\"ops\": {\"adder\": {}, \"op_1\": {}, \"op_2\": {}, \"plus_one\": {}}}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.42ddf3fac74380a93f8508f2a0f6450afb177d35" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", - "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" - } - ], - "given_name": null, - "key": "Shape.360ea318ffe78a111434a5bb7409ef66c9692290", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.42ddf3fac74380a93f8508f2a0f6450afb177d35": { + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", "is_required": false, - "name": "plus_one", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "marker_to_close", + "type_key": "String" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"ops\": {\"adder\": {}, \"op_1\": {}, \"op_2\": {}, \"plus_one\": {}}}", - "description": null, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", "is_required": false, - "name": "subgraph", - "type_key": "Shape.27d4be3152e89c65ecb4ce8c588d8226cc827e0d" + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" } ], "given_name": null, - "key": "Shape.42ddf3fac74380a93f8508f2a0f6450afb177d35", + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -77554,30 +78537,21 @@ "__class__": "ConfigFieldSnap", "default_provided": false, "default_value_as_json_str": null, - "description": "[DEPRECATED]", - "is_required": false, - "name": "marker_to_close", + "description": null, + "is_required": true, + "name": "path", "type_key": "String" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", - "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" } ], "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -77587,20 +78561,20 @@ "default_provided": false, "default_value_as_json_str": null, "description": null, - "is_required": true, - "name": "path", - "type_key": "String" + "is_required": false, + "name": "config", + "type_key": "Any" } ], "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.5403fe5383515d106ffe3bb7f1a927b1cbb4e8a9": { + "Shape.9921ba4c33a841f7499fd90a37710a02cf33b795": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -77608,65 +78582,65 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", "is_required": false, - "name": "adder", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": null, + "description": "Configure how loggers emit messages within a run.", "is_required": false, - "name": "op_1", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"throw_a_thing\": {}}", + "description": "Configure runtime parameters for ops or assets.", "is_required": false, - "name": "op_2", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "ops", + "type_key": "Shape.b3443ab7f0f2bae987a6d21cad9cfe00784f6ccb" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", "is_required": false, - "name": "plus_one", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" } ], "given_name": null, - "key": "Shape.5403fe5383515d106ffe3bb7f1a927b1cbb4e8a9", + "key": "Shape.9921ba4c33a841f7499fd90a37710a02cf33b795", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, "scalar_kind": null, "type_param_keys": null }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "Shape.b3443ab7f0f2bae987a6d21cad9cfe00784f6ccb": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, "fields": [ { "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, + "default_provided": true, + "default_value_as_json_str": "{}", "description": null, "is_required": false, - "name": "config", - "type_key": "Any" + "name": "throw_a_thing", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "key": "Shape.b3443ab7f0f2bae987a6d21cad9cfe00784f6ccb", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -77818,39 +78792,18 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "num", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "subgraph" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "plus_one", - "solid_name": "plus_one", - "tags": {} - }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [], "is_dynamic_mapped": false, - "solid_def_name": "subgraph", - "solid_name": "subgraph", + "solid_def_name": "throw_a_thing", + "solid_name": "throw_a_thing", "tags": {} } ] }, "description": null, - "graph_def_name": "nested_job", + "graph_def_name": "naughty_programmer_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -77889,184 +78842,14 @@ "name": "io_manager" } ], - "root_config_key": "Shape.360ea318ffe78a111434a5bb7409ef66c9692290" + "root_config_key": "Shape.9921ba4c33a841f7499fd90a37710a02cf33b795" } ], - "name": "nested_job", + "name": "naughty_programmer_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", - "composite_solid_def_snaps": [ - { - "__class__": "CompositeSolidDefSnap", - "config_field_snap": null, - "dep_structure_snapshot": { - "__class__": "DependencyStructureSnapshot", - "solid_invocation_snaps": [ - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "num1", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "op_1" - } - ] - }, - { - "__class__": "InputDependencySnap", - "input_name": "num2", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "op_2" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "adder", - "solid_name": "adder", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "op_1", - "solid_name": "op_1", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "op_2", - "solid_name": "op_2", - "tags": {} - }, - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [ - { - "__class__": "InputDependencySnap", - "input_name": "num", - "is_dynamic_collect": false, - "upstream_output_snaps": [ - { - "__class__": "OutputHandleSnap", - "output_name": "result", - "solid_name": "adder" - } - ] - } - ], - "is_dynamic_mapped": false, - "solid_def_name": "plus_one", - "solid_name": "plus_one", - "tags": {} - } - ] - }, - "description": null, - "input_def_snaps": [], - "input_mapping_snaps": [], - "name": "subgraph", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "output_mapping_snaps": [ - { - "__class__": "OutputMappingSnap", - "external_output_name": "result", - "mapped_output_name": "result", - "mapped_solid_name": "plus_one" - } - ], - "tags": {} - } - ], + "composite_solid_def_snaps": [], "solid_def_snaps": [ - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "num1" - }, - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "num2" - } - ], - "name": "adder", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "op_1", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -78080,41 +78863,7 @@ }, "description": null, "input_def_snaps": [], - "name": "op_2", - "output_def_snaps": [ - { - "__class__": "OutputDefSnap", - "dagster_type_key": "Any", - "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - } - ], - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Int", - "description": null, - "name": "num" - } - ], - "name": "plus_one", + "name": "throw_a_thing", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -78135,7 +78884,7 @@ ''' # --- # name: test_all_snapshot_ids[97] - '8e137c24b2245e55025e1cc7b71a42b99425dbec' + '913c310b609478d52a81ee83bdd4b095d0f2932d' # --- # name: test_all_snapshot_ids[98] ''' @@ -78604,6 +79353,111 @@ "scalar_kind": null, "type_param_keys": null }, + "Shape.27d4be3152e89c65ecb4ce8c588d8226cc827e0d": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"adder\": {}, \"op_1\": {}, \"op_2\": {}, \"plus_one\": {}}", + "description": null, + "is_required": false, + "name": "ops", + "type_key": "Shape.5403fe5383515d106ffe3bb7f1a927b1cbb4e8a9" + } + ], + "given_name": null, + "key": "Shape.27d4be3152e89c65ecb4ce8c588d8226cc827e0d", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.360ea318ffe78a111434a5bb7409ef66c9692290": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"plus_one\": {}, \"subgraph\": {\"ops\": {\"adder\": {}, \"op_1\": {}, \"op_2\": {}, \"plus_one\": {}}}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.42ddf3fac74380a93f8508f2a0f6450afb177d35" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"io_manager\": {}}", + "description": "Configure how shared resources are implemented within a run.", + "is_required": false, + "name": "resources", + "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + } + ], + "given_name": null, + "key": "Shape.360ea318ffe78a111434a5bb7409ef66c9692290", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.42ddf3fac74380a93f8508f2a0f6450afb177d35": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "plus_one", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"ops\": {\"adder\": {}, \"op_1\": {}, \"op_2\": {}, \"plus_one\": {}}}", + "description": null, + "is_required": false, + "name": "subgraph", + "type_key": "Shape.27d4be3152e89c65ecb4ce8c588d8226cc827e0d" + } + ], + "given_name": null, + "key": "Shape.42ddf3fac74380a93f8508f2a0f6450afb177d35", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { "__class__": "ConfigTypeSnap", "description": null, @@ -78659,7 +79513,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.5e7f7baf52b7ed6a625b2f172c255b54642d5cf3": { + "Shape.5403fe5383515d106ffe3bb7f1a927b1cbb4e8a9": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -78667,42 +79521,42 @@ { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + "name": "adder", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", + "description": null, "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + "name": "op_1", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"return_foo\": {}, \"return_hello_world\": {}}", - "description": "Configure runtime parameters for ops or assets.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "ops", - "type_key": "Shape.ac8a4423400648ec2ae58a67c8a877953652bf1f" + "name": "op_2", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, { "__class__": "ConfigFieldSnap", "default_provided": true, - "default_value_as_json_str": "{\"io_manager\": {}}", - "description": "Configure how shared resources are implemented within a run.", + "default_value_as_json_str": "{}", + "description": null, "is_required": false, - "name": "resources", - "type_key": "Shape.1578133c1c71e8e3c9cf3ad46c216eb51b48c778" + "name": "plus_one", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" } ], "given_name": null, - "key": "Shape.5e7f7baf52b7ed6a625b2f172c255b54642d5cf3", + "key": "Shape.5403fe5383515d106ffe3bb7f1a927b1cbb4e8a9", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -78732,38 +79586,6 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.ac8a4423400648ec2ae58a67c8a877953652bf1f": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "return_foo", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "return_hello_world", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.ac8a4423400648ec2ae58a67c8a877953652bf1f", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709": { "__class__": "ConfigTypeSnap", "description": null, @@ -78909,39 +79731,39 @@ "dep_structure_snapshot": { "__class__": "DependencyStructureSnapshot", "solid_invocation_snaps": [ - { - "__class__": "SolidInvocationSnap", - "input_dep_snaps": [], - "is_dynamic_mapped": false, - "solid_def_name": "return_foo", - "solid_name": "return_foo", - "tags": {} - }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [ { "__class__": "InputDependencySnap", - "input_name": "_foo", + "input_name": "num", "is_dynamic_collect": false, "upstream_output_snaps": [ { "__class__": "OutputHandleSnap", "output_name": "result", - "solid_name": "return_foo" + "solid_name": "subgraph" } ] } ], "is_dynamic_mapped": false, - "solid_def_name": "return_hello_world", - "solid_name": "return_hello_world", + "solid_def_name": "plus_one", + "solid_name": "plus_one", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "subgraph", + "solid_name": "subgraph", "tags": {} } ] }, "description": null, - "graph_def_name": "no_config_chain_job", + "graph_def_name": "nested_job", "lineage_snapshot": null, "mode_def_snaps": [ { @@ -78980,14 +79802,157 @@ "name": "io_manager" } ], - "root_config_key": "Shape.5e7f7baf52b7ed6a625b2f172c255b54642d5cf3" + "root_config_key": "Shape.360ea318ffe78a111434a5bb7409ef66c9692290" } ], - "name": "no_config_chain_job", + "name": "nested_job", "solid_definitions_snapshot": { "__class__": "SolidDefinitionsSnapshot", - "composite_solid_def_snaps": [], + "composite_solid_def_snaps": [ + { + "__class__": "CompositeSolidDefSnap", + "config_field_snap": null, + "dep_structure_snapshot": { + "__class__": "DependencyStructureSnapshot", + "solid_invocation_snaps": [ + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "num1", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "op_1" + } + ] + }, + { + "__class__": "InputDependencySnap", + "input_name": "num2", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "op_2" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "adder", + "solid_name": "adder", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "op_1", + "solid_name": "op_1", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "op_2", + "solid_name": "op_2", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [ + { + "__class__": "InputDependencySnap", + "input_name": "num", + "is_dynamic_collect": false, + "upstream_output_snaps": [ + { + "__class__": "OutputHandleSnap", + "output_name": "result", + "solid_name": "adder" + } + ] + } + ], + "is_dynamic_mapped": false, + "solid_def_name": "plus_one", + "solid_name": "plus_one", + "tags": {} + } + ] + }, + "description": null, + "input_def_snaps": [], + "input_mapping_snaps": [], + "name": "subgraph", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "output_mapping_snaps": [ + { + "__class__": "OutputMappingSnap", + "external_output_name": "result", + "mapped_output_name": "result", + "mapped_solid_name": "plus_one" + } + ], + "tags": {} + } + ], "solid_def_snaps": [ + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Int", + "description": null, + "name": "num1" + }, + { + "__class__": "InputDefSnap", + "dagster_type_key": "Int", + "description": null, + "name": "num2" + } + ], + "name": "adder", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, { "__class__": "SolidDefSnap", "config_field_snap": { @@ -79001,7 +79966,34 @@ }, "description": null, "input_def_snaps": [], - "name": "return_foo", + "name": "op_1", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "op_2", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -79030,12 +80022,12 @@ "input_def_snaps": [ { "__class__": "InputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "Int", "description": null, - "name": "_foo" + "name": "num" } ], - "name": "return_hello_world", + "name": "plus_one", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -79056,7 +80048,7 @@ ''' # --- # name: test_all_snapshot_ids[99] - 'ab8f4b864ee53d2d9304b85f7a368aad7f678f29' + '8e137c24b2245e55025e1cc7b71a42b99425dbec' # --- # name: test_all_snapshot_ids[9] '9699a524d810d89264bf60d01dab3c751fe47461' diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr index f32dbb64d187d..7f1eb36ec5e46 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr @@ -1233,6 +1233,14 @@ 'handleID': 'hanging_graph.hanging_op', }), }), + dict({ + 'pipeline': dict({ + 'name': 'hanging_partitioned_job', + }), + 'solidHandle': dict({ + 'handleID': 'hanging_op', + }), + }), ]), }), dict({ @@ -1585,6 +1593,14 @@ 'handleID': 'my_op', }), }), + dict({ + 'pipeline': dict({ + 'name': 'hanging_partitioned_job', + }), + 'solidHandle': dict({ + 'handleID': 'my_op', + }), + }), dict({ 'pipeline': dict({ 'name': 'static_partitioned_job', diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py index 2be626cef1edd..81b656daad511 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py @@ -1400,6 +1400,15 @@ def hanging_op(context, my_op): time.sleep(0.1) +@job( + partitions_def=integers_partitions, + config=integers_config, + resource_defs={"hanging_asset_resource": hanging_asset_resource}, +) +def hanging_partitioned_job(): + hanging_op(my_op()) + + @op def never_runs_op(hanging_op): pass @@ -2036,6 +2045,7 @@ def define_standard_jobs() -> Sequence[JobDefinition]: hard_failer, hello_world_with_tags, infinite_loop_job, + hanging_partitioned_job, integers, job_with_default_config, job_with_enum_config, diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py index 8d12e2e0c9f5f..5ae64fce18d1f 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py @@ -19,6 +19,7 @@ execute_asset_backfill_iteration_inner, ) from dagster._core.execution.backfill import BulkActionStatus, PartitionBackfill +from dagster._core.execution.job_backfill import execute_job_backfill_iteration from dagster._core.remote_representation.origin import RemotePartitionSetOrigin from dagster._core.storage.dagster_run import DagsterRun, DagsterRunStatus, RunsFilter from dagster._core.storage.tags import ( @@ -293,6 +294,21 @@ def _execute_backfill_iteration_with_side_effects(graphql_context, backfill_id): ) +def _execute_job_backfill_iteration_with_side_effects(graphql_context, backfill_id): + """Executes a job backfill iteration with side effects (i.e. updates run status and bulk action status).""" + with get_workspace_process_context(graphql_context.instance) as context: + backfill = graphql_context.instance.get_backfill(backfill_id) + list( + execute_job_backfill_iteration( + backfill=backfill, + logger=logging.getLogger("fake_logger"), + workspace_process_context=context, + instance=graphql_context.instance, + debug_crash_flags=None, + ) + ) + + def _mock_asset_backfill_runs( graphql_context, asset_key: AssetKey, @@ -525,7 +541,7 @@ def test_launch_partial_backfill(self, graphql_context): assert len(result.data["partitionBackfillOrError"]["partitionNames"]) == 2 assert result.data["partitionBackfillOrError"]["reexecutionSteps"] == ["after_failure"] - def test_cancel_backfill(self, graphql_context): + def test_cancel_job_backfill(self, graphql_context): repository_selector = infer_repository_selector(graphql_context) result = execute_dagster_graphql( graphql_context, @@ -534,9 +550,9 @@ def test_cancel_backfill(self, graphql_context): "backfillParams": { "selector": { "repositorySelector": repository_selector, - "partitionSetName": "integers_partition_set", + "partitionSetName": "hanging_partitioned_job_partition_set", }, - "partitionNames": ["2", "3"], + "partitionNames": ["1", "2"], } }, ) @@ -556,8 +572,51 @@ def test_cancel_backfill(self, graphql_context): assert result.data assert result.data["partitionBackfillOrError"]["__typename"] == "PartitionBackfill" assert result.data["partitionBackfillOrError"]["status"] == "REQUESTED" - assert result.data["partitionBackfillOrError"]["numCancelable"] == 2 - assert len(result.data["partitionBackfillOrError"]["partitionNames"]) == 2 + + # Update backfill data to update the partition checkpoint, but manually launch the run + # since launching the run via the backfill iteration loop will cause test process will hang forever. + backfill = graphql_context.instance.get_backfill(backfill_id) + partition_to_run = "1" + graphql_context.instance.update_backfill( + backfill.with_partition_checkpoint(partition_to_run) + ) + + # Launch the run that runs forever + selector = infer_job_selector(graphql_context, "hanging_partitioned_job") + with safe_tempfile_path() as path: + result = execute_dagster_graphql( + graphql_context, + LAUNCH_PIPELINE_EXECUTION_MUTATION, + variables={ + "executionParams": { + "selector": selector, + "mode": "default", + "runConfigData": { + "resources": {"hanging_asset_resource": {"config": {"file": path}}} + }, + "executionMetadata": { + "tags": [ + {"key": "dagster/partition", "value": partition_to_run}, + {"key": BACKFILL_ID_TAG, "value": backfill_id}, + ] + }, + } + }, + ) + + assert not result.errors + assert result.data + assert result.data["launchPipelineExecution"]["__typename"] == "LaunchRunSuccess" + + # ensure the execution has happened + start = time.time() + while not os.path.exists(path): + time.sleep(0.1) + assert time.time() - start < 60, "timed out waiting for file" + + runs = graphql_context.instance.get_runs(RunsFilter(tags={BACKFILL_ID_TAG: backfill_id})) + assert len(runs) == 1 + assert runs[0].status == DagsterRunStatus.STARTED result = execute_dagster_graphql( graphql_context, @@ -567,6 +626,25 @@ def test_cancel_backfill(self, graphql_context): assert result.data assert result.data["cancelPartitionBackfill"]["__typename"] == "CancelBackfillSuccess" + result = execute_dagster_graphql( + graphql_context, + PARTITION_PROGRESS_QUERY, + variables={"backfillId": backfill_id}, + ) + assert not result.errors + assert result.data + assert result.data["partitionBackfillOrError"]["__typename"] == "PartitionBackfill" + assert result.data["partitionBackfillOrError"]["status"] == "CANCELING" + + while ( + graphql_context.instance.get_backfill(backfill_id).status != BulkActionStatus.CANCELED + ): + _execute_job_backfill_iteration_with_side_effects(graphql_context, backfill_id) + + runs = graphql_context.instance.get_runs(RunsFilter(tags={BACKFILL_ID_TAG: backfill_id})) + assert len(runs) == 1 + assert runs[0].status == DagsterRunStatus.CANCELED + result = execute_dagster_graphql( graphql_context, PARTITION_PROGRESS_QUERY, @@ -606,6 +684,22 @@ def test_cancel_then_retry_backfill(self, graphql_context): assert result.data assert result.data["cancelPartitionBackfill"]["__typename"] == "CancelBackfillSuccess" + # run the backfill iteration until the backfill is canceled + while ( + graphql_context.instance.get_backfill(backfill_id).status != BulkActionStatus.CANCELED + ): + _execute_job_backfill_iteration_with_side_effects(graphql_context, backfill_id) + + result = execute_dagster_graphql( + graphql_context, + PARTITION_PROGRESS_QUERY, + variables={"backfillId": backfill_id}, + ) + assert not result.errors + assert result.data + assert result.data["partitionBackfillOrError"]["__typename"] == "PartitionBackfill" + assert result.data["partitionBackfillOrError"]["status"] == "CANCELED" + result = execute_dagster_graphql( graphql_context, RETRY_BACKFILL_MUTATION, diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py index f33fc6139d7e7..e56f220535e71 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py @@ -1,9 +1,13 @@ from typing import Any +from unittest.mock import patch from dagster._core.test_utils import wait_for_runs_to_finish from dagster._core.workspace.context import WorkspaceRequestContext -from dagster_graphql.client.query import LAUNCH_PIPELINE_EXECUTION_MUTATION -from dagster_graphql.test.utils import execute_dagster_graphql, infer_job_selector +from dagster_graphql.client.query import ( + LAUNCH_MULTIPLE_RUNS_MUTATION, + LAUNCH_PIPELINE_EXECUTION_MUTATION, +) +from dagster_graphql.test.utils import GqlResult, execute_dagster_graphql, infer_job_selector from dagster_graphql_tests.graphql.graphql_context_test_suite import ( GraphQLContextVariant, @@ -32,6 +36,12 @@ BaseTestSuite: Any = make_graphql_context_test_suite( context_variants=GraphQLContextVariant.all_executing_variants() ) +LaunchFailTestSuite: Any = make_graphql_context_test_suite( + context_variants=GraphQLContextVariant.all_non_launchable_variants() +) +ReadOnlyTestSuite: Any = make_graphql_context_test_suite( + context_variants=GraphQLContextVariant.all_readonly_variants() +) class TestBasicLaunch(BaseTestSuite): @@ -83,10 +93,99 @@ def test_run_launcher_subset(self, graphql_context: WorkspaceRequestContext): assert result.data["pipelineRunOrError"]["status"] == "SUCCESS" assert result.data["pipelineRunOrError"]["stats"]["stepsSucceeded"] == 1 + def test_run_launcher_unauthorized(self, graphql_context: WorkspaceRequestContext): + selector = infer_job_selector(graphql_context, "no_config_job") -LaunchFailTestSuite: Any = make_graphql_context_test_suite( - context_variants=GraphQLContextVariant.all_non_launchable_variants() -) + with patch.object(graphql_context, "has_permission_for_location", return_value=False): + with patch.object(graphql_context, "was_permission_checked", return_value=True): + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_PIPELINE_EXECUTION_MUTATION, + variables={"executionParams": {"selector": selector, "mode": "default"}}, + ) + assert result.data["launchPipelineExecution"]["__typename"] == "UnauthorizedError" + + +class TestMultipleLaunch(BaseTestSuite): + def test_multiple_run_launcher_same_job(self, graphql_context: WorkspaceRequestContext): + selector = infer_job_selector(graphql_context, "no_config_job") + + # test with multiple of the same job + executionParamsList = [ + {"selector": selector, "mode": "default"}, + {"selector": selector, "mode": "default"}, + {"selector": selector, "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + launches = result.data["launchMultipleRuns"] + + assert launches["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in launches + results = launches["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "LaunchRunSuccess" + + def test_multiple_run_launcher_multiple_jobs(self, graphql_context: WorkspaceRequestContext): + selectors = [ + infer_job_selector(graphql_context, "no_config_job"), + infer_job_selector(graphql_context, "more_complicated_config", ["noop_op"]), + ] + + # test with multiple of the same job + executionParamsList = [ + {"selector": selectors[0], "mode": "default"}, + {"selector": selectors[1], "mode": "default"}, + {"selector": selectors[0], "mode": "default"}, + {"selector": selectors[1], "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + launches = result.data["launchMultipleRuns"] + + assert launches["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in launches + results = launches["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "LaunchRunSuccess" + + def test_multiple_launch_failure_unauthorized(self, graphql_context: WorkspaceRequestContext): + executionParamsList = [ + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + ] + + # mock no permissions + with patch.object(graphql_context, "has_permission_for_location", return_value=False): + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in result_data + + results = result_data["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "UnauthorizedError" class TestFailedLaunch(LaunchFailTestSuite): @@ -97,6 +196,7 @@ def test_launch_failure(self, graphql_context: WorkspaceRequestContext): query=LAUNCH_PIPELINE_EXECUTION_MUTATION, variables={"executionParams": {"selector": selector, "mode": "default"}}, ) + assert result.data["launchPipelineExecution"]["__typename"] != "LaunchRunSuccess" # fetch the most recent run, which should be this one that just failed to launch @@ -105,7 +205,140 @@ def test_launch_failure(self, graphql_context: WorkspaceRequestContext): result = execute_dagster_graphql( context=graphql_context, query=RUN_QUERY, variables={"runId": run.run_id} ) + assert result.data["pipelineRunOrError"]["__typename"] == "Run" assert result.data["pipelineRunOrError"]["status"] == "FAILURE" assert result.data["pipelineRunOrError"]["startTime"] assert result.data["pipelineRunOrError"]["endTime"] + + +class TestFailedMultipleLaunch(LaunchFailTestSuite): + def test_multiple_launch_failure(self, graphql_context: WorkspaceRequestContext): + executionParamsList = [ + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + results = result_data["launchMultipleRunsResult"] + + assert len(results) == 2 + + for run_result in results: + assert run_result["__typename"] == "PythonError" + assert run_result["message"].startswith( + "NotImplementedError: The entire purpose of this is to throw on launch" + ) + assert run_result["className"] == "NotImplementedError" + + +class TestFailedMultipleLaunchReadOnly(ReadOnlyTestSuite): + def test_multiple_launch_failure_readonly(self, graphql_context: WorkspaceRequestContext): + executionParamsList = [ + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in result_data + + results = result_data["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "UnauthorizedError" + + +class TestSuccessAndFailureMultipleLaunch(BaseTestSuite): + def test_launch_multiple_runs_success_and_failure( + self, graphql_context: WorkspaceRequestContext + ): + launchSuccessExecutionParams = [ + { + "selector": { + "repositoryLocationName": "test_location", + "repositoryName": "test_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + { + "selector": { + "repositoryLocationName": "test_location", + "repositoryName": "test_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + ] + + pipelineNotFoundExecutionParams = [ + { + "selector": { + "repositoryLocationName": "test_location", + "repositoryName": "test_dict_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + { + "selector": { + "repositoryLocationName": "test_location", + "repositoryName": "test_dict_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + ] + + executionParamsList = [executionParams for executionParams in launchSuccessExecutionParams] + executionParamsList.extend( + [executionParams for executionParams in pipelineNotFoundExecutionParams] + ) + + result: GqlResult = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + results = result_data["launchMultipleRunsResult"] + + assert len(results) == 4 + + assert results[0]["__typename"] == "LaunchRunSuccess" + assert results[1]["__typename"] == "LaunchRunSuccess" + assert results[2]["__typename"] == "PipelineNotFoundError" + assert results[3]["__typename"] == "PipelineNotFoundError" diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 2e3e391b6d063..8c5ee3805c460 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -402,6 +402,7 @@ DefaultSensorStatus as DefaultSensorStatus, SensorDefinition as SensorDefinition, SensorEvaluationContext as SensorEvaluationContext, + SensorReturnTypesUnion as SensorReturnTypesUnion, build_sensor_context as build_sensor_context, ) from dagster._core.definitions.source_asset import SourceAsset as SourceAsset diff --git a/python_modules/dagster/dagster/_cli/project.py b/python_modules/dagster/dagster/_cli/project.py index 1fa5a217a3310..d9c582804e6cb 100644 --- a/python_modules/dagster/dagster/_cli/project.py +++ b/python_modules/dagster/dagster/_cli/project.py @@ -63,7 +63,7 @@ def check_if_pypi_package_conflict_exists(project_name: str) -> PackageConflictC """ if any(keyword in project_name for keyword in FLAGGED_PACKAGE_KEYWORDS): try: - res = requests.get(f"https://pypi.org/pypi/{project_name}") + res = requests.get(f"https://pypi.org/pypi/{project_name}/json") if res.status_code == 200: return PackageConflictCheckResult(request_error_msg=None, conflict_exists=True) except Exception as e: diff --git a/python_modules/dagster/dagster/_core/definitions/asset_sensor_definition.py b/python_modules/dagster/dagster/_core/definitions/asset_sensor_definition.py index c9b48ce10951c..0a7667a52727d 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_sensor_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_sensor_definition.py @@ -9,8 +9,8 @@ from dagster._core.definitions.run_request import RunRequest, SkipReason from dagster._core.definitions.sensor_definition import ( DefaultSensorStatus, - RawSensorEvaluationFunctionReturn, SensorDefinition, + SensorReturnTypesUnion, SensorType, validate_and_get_resource_dict, ) @@ -80,7 +80,7 @@ def __init__( job_name: Optional[str], asset_materialization_fn: Callable[ ..., - RawSensorEvaluationFunctionReturn, + SensorReturnTypesUnion, ], minimum_interval_seconds: Optional[int] = None, description: Optional[str] = None, diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/op_decorator.py b/python_modules/dagster/dagster/_core/definitions/decorators/op_decorator.py index 21fe0250cc8f7..a30ba32e52954 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/op_decorator.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/op_decorator.py @@ -446,8 +446,8 @@ def resolve_checked_op_fn_inputs( ) raise DagsterInvalidDefinitionError( f"{decorator_name} '{fn_name}' decorated function does not have argument(s)" - f" '{undeclared_inputs_printed}'. {decorator_name}-decorated functions should have a" - f" keyword argument for each of their Ins{nothing_exemption}. Alternatively, they can" + f" '{undeclared_inputs_printed}'. {decorator_name}-decorated functions should have an" + f" argument for each of their Ins{nothing_exemption}. Alternatively, they can" " accept **kwargs." ) diff --git a/python_modules/dagster/dagster/_core/definitions/run_status_sensor_definition.py b/python_modules/dagster/dagster/_core/definitions/run_status_sensor_definition.py index 793c186012ce2..05bc928f2b084 100644 --- a/python_modules/dagster/dagster/_core/definitions/run_status_sensor_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/run_status_sensor_definition.py @@ -31,11 +31,11 @@ from dagster._core.definitions.sensor_definition import ( DagsterRunReaction, DefaultSensorStatus, - RawSensorEvaluationFunctionReturn, RunRequest, SensorDefinition, SensorEvaluationContext, SensorResult, + SensorReturnTypesUnion, SensorType, SkipReason, get_context_param_name, @@ -71,12 +71,12 @@ ) RunStatusSensorEvaluationFunction: TypeAlias = Union[ - Callable[..., RawSensorEvaluationFunctionReturn], - Callable[..., RawSensorEvaluationFunctionReturn], + Callable[..., SensorReturnTypesUnion], + Callable[..., SensorReturnTypesUnion], ] RunFailureSensorEvaluationFn: TypeAlias = Union[ - Callable[..., RawSensorEvaluationFunctionReturn], - Callable[..., RawSensorEvaluationFunctionReturn], + Callable[..., SensorReturnTypesUnion], + Callable[..., SensorReturnTypesUnion], ] @@ -980,7 +980,7 @@ def _wrapped_fn( metadata=metadata, ) - def __call__(self, *args, **kwargs) -> RawSensorEvaluationFunctionReturn: + def __call__(self, *args, **kwargs) -> SensorReturnTypesUnion: context_param_name = get_context_param_name(self._run_status_sensor_fn) context = get_or_create_sensor_context( self._run_status_sensor_fn, diff --git a/python_modules/dagster/dagster/_core/definitions/sensor_definition.py b/python_modules/dagster/dagster/_core/definitions/sensor_definition.py index eb73a31258bdd..4331a541550c4 100644 --- a/python_modules/dagster/dagster/_core/definitions/sensor_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/sensor_definition.py @@ -463,7 +463,7 @@ def log_key(self) -> Optional[Sequence[str]]: return self._log_key -RawSensorEvaluationFunctionReturn = Union[ +SensorReturnTypesUnion: TypeAlias = Union[ Iterator[Union[SkipReason, RunRequest, DagsterRunReaction, SensorResult]], Sequence[RunRequest], SkipReason, @@ -472,7 +472,7 @@ def log_key(self) -> Optional[Sequence[str]]: SensorResult, None, ] -RawSensorEvaluationFunction: TypeAlias = Callable[..., RawSensorEvaluationFunctionReturn] +RawSensorEvaluationFunction: TypeAlias = Callable[..., SensorReturnTypesUnion] SensorEvaluationFunction: TypeAlias = Callable[ ..., Sequence[Union[None, SensorResult, SkipReason, RunRequest]] @@ -787,7 +787,7 @@ def dagster_internal_init( target=target, ) - def __call__(self, *args, **kwargs) -> RawSensorEvaluationFunctionReturn: + def __call__(self, *args, **kwargs) -> SensorReturnTypesUnion: context_param_name_if_present = get_context_param_name(self._raw_fn) context = get_or_create_sensor_context(self._raw_fn, *args, **kwargs) diff --git a/python_modules/dagster/dagster/_core/execution/asset_backfill.py b/python_modules/dagster/dagster/_core/execution/asset_backfill.py index b9e4cdc18094f..82eedb6e5eb04 100644 --- a/python_modules/dagster/dagster/_core/execution/asset_backfill.py +++ b/python_modules/dagster/dagster/_core/execution/asset_backfill.py @@ -51,13 +51,7 @@ from dagster._core.event_api import AssetRecordsFilter from dagster._core.execution.submit_asset_runs import submit_asset_run from dagster._core.instance import DagsterInstance, DynamicPartitionsStore -from dagster._core.storage.dagster_run import ( - CANCELABLE_RUN_STATUSES, - IN_PROGRESS_RUN_STATUSES, - NOT_FINISHED_STATUSES, - DagsterRunStatus, - RunsFilter, -) +from dagster._core.storage.dagster_run import NOT_FINISHED_STATUSES, DagsterRunStatus, RunsFilter from dagster._core.storage.tags import ( ASSET_PARTITION_RANGE_END_TAG, ASSET_PARTITION_RANGE_START_TAG, @@ -80,8 +74,6 @@ def get_asset_backfill_run_chunk_size(): MATERIALIZATION_CHUNK_SIZE = 1000 -MAX_RUNS_CANCELED_PER_ITERATION = 50 - class AssetBackfillStatus(Enum): IN_PROGRESS = "IN_PROGRESS" @@ -1161,44 +1153,17 @@ def execute_asset_backfill_iteration( ) elif backfill.status == BulkActionStatus.CANCELING: - if not instance.run_coordinator: - check.failed("The instance must have a run coordinator in order to cancel runs") - - # Query for cancelable runs, enforcing a limit on the number of runs to cancel in an iteration - # as canceling runs incurs cost - runs_to_cancel_in_iteration = instance.run_storage.get_run_ids( - filters=RunsFilter( - statuses=CANCELABLE_RUN_STATUSES, - tags={ - BACKFILL_ID_TAG: backfill.backfill_id, - }, - ), - limit=MAX_RUNS_CANCELED_PER_ITERATION, - ) + from dagster._core.execution.backfill import cancel_backfill_runs_and_cancellation_complete - yield None + for all_runs_canceled in cancel_backfill_runs_and_cancellation_complete( + instance=instance, backfill_id=backfill.backfill_id + ): + yield None - waiting_for_runs_to_finish_after_cancelation = False - if runs_to_cancel_in_iteration: - for run_id in runs_to_cancel_in_iteration: - instance.run_coordinator.cancel_run(run_id) - yield None - else: - # Check at the beginning of the tick whether there are any runs that we are still - # waiting to move into a terminal state. If there are none and the backfill data is - # still missing partitions at the end of the tick, that indicates a framework problem. - # (It's important that we check for these runs before updating the backfill data - - # if we did them in the reverse order, a run that finishes between the two checks - # might not be incorporated into the backfill data, causing us to incorrectly decide - # there was a framework error. - run_waiting_to_cancel = instance.get_run_ids( - RunsFilter( - tags={BACKFILL_ID_TAG: backfill.backfill_id}, - statuses=IN_PROGRESS_RUN_STATUSES, - ), - limit=1, + if not isinstance(all_runs_canceled, bool): + check.failed( + "Expected cancel_backfill_runs_and_cancellation_complete to return a boolean" ) - waiting_for_runs_to_finish_after_cancelation = len(run_waiting_to_cancel) > 0 # Update the asset backfill data to contain the newly materialized/failed partitions. updated_asset_backfill_data = None @@ -1235,11 +1200,7 @@ def execute_asset_backfill_iteration( instance.update_backfill(updated_backfill) - if ( - len(runs_to_cancel_in_iteration) == 0 - and not all_partitions_marked_completed - and not waiting_for_runs_to_finish_after_cancelation - ): + if all_runs_canceled and not all_partitions_marked_completed: check.failed( "All runs have completed, but not all requested partitions have been marked as materialized or failed. " "This is likely a system error. Please report this issue to the Dagster team." diff --git a/python_modules/dagster/dagster/_core/execution/backfill.py b/python_modules/dagster/dagster/_core/execution/backfill.py index 2eac210ece544..b178371512e4a 100644 --- a/python_modules/dagster/dagster/_core/execution/backfill.py +++ b/python_modules/dagster/dagster/_core/execution/backfill.py @@ -1,6 +1,6 @@ from datetime import datetime from enum import Enum -from typing import Mapping, NamedTuple, Optional, Sequence, Union +from typing import TYPE_CHECKING, Iterator, Mapping, NamedTuple, Optional, Sequence, Union from dagster import _check as check from dagster._core.definitions import AssetKey @@ -20,12 +20,22 @@ from dagster._core.instance import DynamicPartitionsStore from dagster._core.remote_representation.external_data import job_name_for_partition_set_snap_name from dagster._core.remote_representation.origin import RemotePartitionSetOrigin -from dagster._core.storage.tags import USER_TAG +from dagster._core.storage.dagster_run import ( + CANCELABLE_RUN_STATUSES, + NOT_FINISHED_STATUSES, + RunsFilter, +) +from dagster._core.storage.tags import BACKFILL_ID_TAG, USER_TAG from dagster._core.workspace.context import BaseWorkspaceRequestContext from dagster._record import record from dagster._serdes import whitelist_for_serdes from dagster._utils.error import SerializableErrorInfo +if TYPE_CHECKING: + from dagster._core.instance import DagsterInstance + +MAX_RUNS_CANCELED_PER_ITERATION = 50 + @whitelist_for_serdes class BulkActionStatus(Enum): @@ -508,3 +518,50 @@ def from_asset_graph_subset( title=title, description=description, ) + + +def cancel_backfill_runs_and_cancellation_complete( + instance: "DagsterInstance", backfill_id: str +) -> Iterator[Union[None, bool]]: + """Cancels MAX_RUNS_CANCELED_PER_ITERATION runs associated with the backfill_id. Ensures that + all runs for the backfill are in a terminal state before indicating that the backfill can be marked + CANCELED. + Yields a boolean indicating the backfill can be considered canceled (ie all runs are canceled). + """ + if not instance.run_coordinator: + check.failed("The instance must have a run coordinator in order to cancel runs") + + # Query for cancelable runs, enforcing a limit on the number of runs to cancel in an iteration + # as canceling runs incurs cost + runs_to_cancel_in_iteration = instance.run_storage.get_run_ids( + filters=RunsFilter( + statuses=CANCELABLE_RUN_STATUSES, + tags={ + BACKFILL_ID_TAG: backfill_id, + }, + ), + limit=MAX_RUNS_CANCELED_PER_ITERATION, + ) + + yield None + + if runs_to_cancel_in_iteration: + # since we are canceling some runs in this iteration, we know that there is more work to do. + # Either cancelling more runs, or waiting for the canceled runs to get to a terminal state + work_done = False + for run_id in runs_to_cancel_in_iteration: + instance.run_coordinator.cancel_run(run_id) + yield None + else: + # If there are no runs to cancel, check if there are any runs still in progress. If there are, + # then we want to wait for them to reach a terminal state before the backfill is marked CANCELED. + run_waiting_to_cancel = instance.get_run_ids( + RunsFilter( + tags={BACKFILL_ID_TAG: backfill_id}, + statuses=NOT_FINISHED_STATUSES, + ), + limit=1, + ) + work_done = len(run_waiting_to_cancel) == 0 + + yield work_done diff --git a/python_modules/dagster/dagster/_core/execution/job_backfill.py b/python_modules/dagster/dagster/_core/execution/job_backfill.py index 56b2764a02a19..480f645cf7081 100644 --- a/python_modules/dagster/dagster/_core/execution/job_backfill.py +++ b/python_modules/dagster/dagster/_core/execution/job_backfill.py @@ -7,7 +7,11 @@ from dagster._core.definitions.partition_key_range import PartitionKeyRange from dagster._core.definitions.selector import JobSubsetSelector from dagster._core.errors import DagsterBackfillFailedError, DagsterInvariantViolationError -from dagster._core.execution.backfill import BulkActionStatus, PartitionBackfill +from dagster._core.execution.backfill import ( + BulkActionStatus, + PartitionBackfill, + cancel_backfill_runs_and_cancellation_complete, +) from dagster._core.execution.plan.resume_retry import ReexecutionStrategy from dagster._core.execution.plan.state import KnownExecutionState from dagster._core.instance import DagsterInstance @@ -58,10 +62,25 @@ def execute_job_backfill_iteration( partition_set = _get_partition_set(workspace_process_context, backfill) + # refetch in case the backfill status has changed + backfill = cast(PartitionBackfill, instance.get_backfill(backfill.backfill_id)) + if backfill.status == BulkActionStatus.CANCELING: + for all_runs_canceled in cancel_backfill_runs_and_cancellation_complete( + instance=instance, backfill_id=backfill.backfill_id + ): + yield None + + if not isinstance(all_runs_canceled, bool): + check.failed( + "Expected cancel_backfill_runs_and_cancellation_complete to return a boolean" + ) + + if all_runs_canceled: + instance.update_backfill(backfill.with_status(BulkActionStatus.CANCELED)) + return + has_more = True while has_more: - # refetch in case the backfill status has changed - backfill = cast(PartitionBackfill, instance.get_backfill(backfill.backfill_id)) if backfill.status != BulkActionStatus.REQUESTED: break diff --git a/python_modules/dagster/dagster/_core/pipes/context.py b/python_modules/dagster/dagster/_core/pipes/context.py index f8dc2788ef1cb..a0533ce3379f9 100644 --- a/python_modules/dagster/dagster/_core/pipes/context.py +++ b/python_modules/dagster/dagster/_core/pipes/context.py @@ -1,4 +1,3 @@ -import os import sys from contextlib import contextmanager from dataclasses import dataclass, field @@ -329,56 +328,7 @@ class PipesSession: @cached_property def default_remote_invocation_info(self) -> Dict[str, str]: - """Key-value pairs encoding metadata about the launching Dagster process, typically attached to the remote - environment. - - Remote execution environments commonly have their own concepts of tags or labels. It's useful to include - Dagster-specific metadata in these environments to help with debugging, monitoring, and linking remote - resources back to Dagster. For example, the Kubernetes Pipes client is using these tags as Kubernetes labels. - - By default the tags include: - * dagster/run-id - * dagster/job - - And, if available: - * dagster/code-location - * dagster/user - * dagster/partition-key - - And, for Dagster+ deployments: - * dagster/deployment-name - * dagster/git-repo - * dagster/git-branch - * dagster/git-sha - """ - tags = { - "dagster/run-id": self.context.run_id, - "dagster/job": self.context.job_name, - } - - if self.context.dagster_run.remote_job_origin: - tags["dagster/code-location"] = ( - self.context.dagster_run.remote_job_origin.repository_origin.code_location_origin.location_name - ) - - if user := self.context.get_tag("dagster/user"): - tags["dagster/user"] = user - - if self.context.has_partition_key: - tags["dagster/partition-key"] = self.context.partition_key - - # now using the walrus operator for os.getenv("DAGSTER_CLOUD_DEPLOYMENT_NAME") - - for env_var, tag in { - "DAGSTER_CLOUD_DEPLOYMENT_NAME": "deployment-name", - "DAGSTER_CLOUD_GIT_REPO": "git-repo", - "DAGSTER_CLOUD_GIT_BRANCH": "git-branch", - "DAGSTER_CLOUD_GIT_SHA": "git-sha", - }.items(): - if value := os.getenv(env_var): - tags[f"dagster/{tag}"] = value - - return tags + return {**self.context.dagster_run.dagster_execution_info} @public def get_bootstrap_env_vars(self) -> Mapping[str, str]: diff --git a/python_modules/dagster/dagster/_core/storage/dagster_run.py b/python_modules/dagster/dagster/_core/storage/dagster_run.py index 5c948829638eb..44a9217b75d46 100644 --- a/python_modules/dagster/dagster/_core/storage/dagster_run.py +++ b/python_modules/dagster/dagster/_core/storage/dagster_run.py @@ -1,5 +1,7 @@ +import os from datetime import datetime from enum import Enum +from functools import cached_property from typing import ( TYPE_CHECKING, AbstractSet, @@ -431,6 +433,56 @@ def get_root_run_id(self) -> Optional[str]: def get_parent_run_id(self) -> Optional[str]: return self.tags.get(PARENT_RUN_ID_TAG) + @cached_property + def dagster_execution_info(self) -> Mapping[str, str]: + """Key-value pairs encoding metadata about the current Dagster run, typically attached to external execution resources. + + Remote execution environments commonly have their own concepts of tags or labels. It's useful to include + Dagster-specific metadata in these environments to help with debugging, monitoring, and linking remote + resources back to Dagster. For example, the Kubernetes Executor and Pipes client are using these tags as Kubernetes labels. + + By default the tags include: + * dagster/run-id + * dagster/job + + And, if available: + * dagster/partition + * dagster/code-location + * dagster/user + + And, for Dagster+ deployments: + * dagster/deployment-name + * dagster/git-repo + * dagster/git-branch + * dagster/git-sha + """ + tags = { + "dagster/run-id": self.run_id, + "dagster/job": self.job_name, + } + + if self.remote_job_origin: + tags["dagster/code-location"] = ( + self.remote_job_origin.repository_origin.code_location_origin.location_name + ) + + if user := self.tags.get("dagster/user"): + tags["dagster/user"] = user + + if partition := self.tags.get("dagster/partition"): + tags["dagster/partition"] = partition + + for env_var, tag in { + "DAGSTER_CLOUD_DEPLOYMENT_NAME": "deployment-name", + "DAGSTER_CLOUD_GIT_REPO": "git-repo", + "DAGSTER_CLOUD_GIT_BRANCH": "git-branch", + "DAGSTER_CLOUD_GIT_SHA": "git-sha", + }.items(): + if value := os.getenv(env_var): + tags[f"dagster/{tag}"] = value + + return tags + def tags_for_storage(self) -> Mapping[str, str]: repository_tags = {} if self.remote_job_origin: diff --git a/python_modules/dagster/dagster/_core/storage/event_log/sqlite/consolidated_sqlite_event_log.py b/python_modules/dagster/dagster/_core/storage/event_log/sqlite/consolidated_sqlite_event_log.py index 24fe196036003..ca7361db8d3bf 100644 --- a/python_modules/dagster/dagster/_core/storage/event_log/sqlite/consolidated_sqlite_event_log.py +++ b/python_modules/dagster/dagster/_core/storage/event_log/sqlite/consolidated_sqlite_event_log.py @@ -113,7 +113,9 @@ def index_connection(self): def has_table(self, table_name: str) -> bool: engine = create_engine(self._conn_string, poolclass=NullPool) - return bool(engine.dialect.has_table(engine.connect(), table_name)) + with engine.connect() as conn: + has_table = bool(engine.dialect.has_table(conn, table_name)) + return has_table def get_db_path(self): return os.path.join(self._base_dir, f"{SQLITE_EVENT_LOG_FILENAME}.db") diff --git a/python_modules/dagster/dagster/_core/storage/tags.py b/python_modules/dagster/dagster/_core/storage/tags.py index 7c22457b39ee5..4b8db616409d3 100644 --- a/python_modules/dagster/dagster/_core/storage/tags.py +++ b/python_modules/dagster/dagster/_core/storage/tags.py @@ -93,7 +93,7 @@ USER_EDITABLE_SYSTEM_TAGS = [ PRIORITY_TAG, MAX_RETRIES_TAG, - RETRY_STRATEGY_TAG, + RESUME_RETRY_TAG, MAX_RUNTIME_SECONDS_TAG, RUN_ISOLATION_TAG, RETRY_ON_ASSET_OR_OP_FAILURE_TAG, @@ -114,6 +114,8 @@ TAGS_TO_MAYBE_OMIT_ON_RETRY = { *RUN_METRIC_TAGS, RUN_FAILURE_REASON_TAG, + RETRY_NUMBER_TAG, + RETRY_STRATEGY_TAG, WILL_RETRY_TAG, AUTO_RETRY_RUN_ID_TAG, *BACKFILL_TAGS, diff --git a/python_modules/dagster/dagster/_daemon/auto_run_reexecution/auto_run_reexecution.py b/python_modules/dagster/dagster/_daemon/auto_run_reexecution/auto_run_reexecution.py index f347ed4829c89..d9a1333edb1d2 100644 --- a/python_modules/dagster/dagster/_daemon/auto_run_reexecution/auto_run_reexecution.py +++ b/python_modules/dagster/dagster/_daemon/auto_run_reexecution/auto_run_reexecution.py @@ -1,19 +1,21 @@ import sys -from typing import Iterator, Optional, Sequence, Tuple, cast +from typing import Iterator, Optional, Sequence, cast +import dagster._check as check from dagster._core.definitions.metadata import MetadataValue from dagster._core.definitions.selector import JobSubsetSelector from dagster._core.events import EngineEventData, RunFailureReason from dagster._core.execution.plan.resume_retry import ReexecutionStrategy +from dagster._core.execution.retries import auto_reexecution_should_retry_run from dagster._core.instance import DagsterInstance from dagster._core.storage.dagster_run import DagsterRun, DagsterRunStatus, RunRecord from dagster._core.storage.tags import ( AUTO_RETRY_RUN_ID_TAG, - MAX_RETRIES_TAG, RETRY_NUMBER_TAG, RETRY_ON_ASSET_OR_OP_FAILURE_TAG, RETRY_STRATEGY_TAG, RUN_FAILURE_REASON_TAG, + WILL_RETRY_TAG, ) from dagster._core.workspace.context import IWorkspaceProcessContext from dagster._daemon.utils import DaemonErrorCapture @@ -22,68 +24,62 @@ DEFAULT_REEXECUTION_POLICY = ReexecutionStrategy.FROM_FAILURE -def filter_runs_to_should_retry( - runs: Sequence[DagsterRun], instance: DagsterInstance, default_max_retries: int -) -> Iterator[Tuple[DagsterRun, int]]: - """Return only runs that should retry along with their retry number (1st retry, 2nd, etc.).""" - - def get_retry_number(run: DagsterRun) -> Optional[int]: - if run.status != DagsterRunStatus.FAILURE: - return None - - raw_max_retries_tag = run.tags.get(MAX_RETRIES_TAG) - if raw_max_retries_tag is None: - max_retries = default_max_retries +def should_retry(run: DagsterRun, instance: DagsterInstance) -> bool: + will_retry_tag_value = run.tags.get(WILL_RETRY_TAG) + if will_retry_tag_value is None: + # If the run doesn't have the WILL_RETRY_TAG, and the run is failed, we + # recalculate if the run should be retried to ensure backward compatibilty + if run.status == DagsterRunStatus.FAILURE: + should_retry_run = auto_reexecution_should_retry_run(instance, run) + # add the tag to the run so that it can be used in other parts of the system + instance.add_run_tags(run.run_id, {WILL_RETRY_TAG: str(should_retry_run).lower()}) else: - try: - max_retries = int(raw_max_retries_tag) - except ValueError: - instance.report_engine_event( - f"Error parsing int from tag {MAX_RETRIES_TAG}, won't retry the run.", run - ) - return None - - if max_retries == 0: - return None - - # TODO: group these to reduce db calls - run_group = instance.get_run_group(run.run_id) - - if run_group: - _, run_group_iter = run_group - run_group_list = list(run_group_iter) - - # Has the parent run already been retried the maximum number of times? (Group includes the parent) - if len(run_group_list) >= max_retries + 1: - return None - - # Does this run already have a child run? - if any([run.run_id == run_.parent_run_id for run_ in run_group_list]): - return None - return len(run_group_list) - else: - return 1 - - default_retry_on_asset_or_op_failure: bool = instance.run_retries_retry_on_asset_or_op_failure + # run is not failed, and shouldn't be retried + return False + else: + should_retry_run = get_boolean_tag_value(will_retry_tag_value, default_value=False) - for run in runs: - retry_number = get_retry_number(run) + if should_retry_run: + return should_retry_run + else: retry_on_asset_or_op_failure = get_boolean_tag_value( run.tags.get(RETRY_ON_ASSET_OR_OP_FAILURE_TAG), - default_value=default_retry_on_asset_or_op_failure, + default_value=instance.run_retries_retry_on_asset_or_op_failure, ) - if retry_number is not None: - if ( - run.tags.get(RUN_FAILURE_REASON_TAG) == RunFailureReason.STEP_FAILURE.value - and not retry_on_asset_or_op_failure - ): - instance.report_engine_event( - "Not retrying run since it failed due to an asset or op failure and run retries " - "are configured with retry_on_asset_or_op_failure set to false.", - run, - ) - else: - yield (run, retry_number) + if ( + run.tags.get(RUN_FAILURE_REASON_TAG) == RunFailureReason.STEP_FAILURE.value + and not retry_on_asset_or_op_failure + ): + instance.report_engine_event( + "Not retrying run since it failed due to an asset or op failure and run retries " + "are configured with retry_on_asset_or_op_failure set to false.", + run, + ) + return False + + +def filter_runs_to_should_retry( + runs: Sequence[DagsterRun], instance: DagsterInstance +) -> Iterator[DagsterRun]: + """Return only runs that should retry along with their retry number (1st retry, 2nd, etc.).""" + for run in runs: + if should_retry(run, instance): + yield run + + +def get_automatically_retried_run_if_exists( + instance: DagsterInstance, run: DagsterRun, run_group: Sequence[DagsterRun] +) -> Optional[DagsterRun]: + if run.tags.get(AUTO_RETRY_RUN_ID_TAG) is not None: + return instance.get_run_by_id(run.tags[AUTO_RETRY_RUN_ID_TAG]) + child_run = next( + (retried_run for retried_run in run_group if run.run_id == retried_run.parent_run_id), None + ) + if child_run is not None and child_run.tags.get(RETRY_NUMBER_TAG) is not None: + # We use the presense of RETRY_NUMBER_TAG to confirm that the child run was launched + # by the automatic retry daemon. If the child run was launched by the user, the tag + # should not be present. + return child_run def get_reexecution_strategy( @@ -104,12 +100,10 @@ def get_reexecution_strategy( def retry_run( failed_run: DagsterRun, - retry_number: int, workspace_context: IWorkspaceProcessContext, ) -> None: """Submit a retry as a re-execute from failure.""" instance = workspace_context.instance - tags = {RETRY_NUMBER_TAG: str(retry_number)} workspace = workspace_context.create_request_context() if not failed_run.remote_job_origin: instance.report_engine_event( @@ -152,8 +146,29 @@ def retry_run( ) ) - strategy = get_reexecution_strategy(failed_run, instance) or DEFAULT_REEXECUTION_POLICY + _, run_group = check.not_none(instance.get_run_group(failed_run.run_id)) + run_group_list = list(run_group) + + # it is possible for the daemon to die between creating the run and submitting it. We account for this + # possibility by checking if the a run already exists in the run group with the parent run id of the + # failed run and resubmit it if necessary. + existing_retried_run = get_automatically_retried_run_if_exists( + instance=instance, run=failed_run, run_group=run_group_list + ) + if existing_retried_run is not None: + # ensure the failed_run has the AUTO_RETRY_RUN_ID_TAG set + if failed_run.tags.get(AUTO_RETRY_RUN_ID_TAG) is None: + instance.add_run_tags( + failed_run.run_id, {AUTO_RETRY_RUN_ID_TAG: existing_retried_run.run_id} + ) + if existing_retried_run.status == DagsterRunStatus.NOT_STARTED: + # A run already exists but was not submitted. + instance.submit_run(existing_retried_run.run_id, workspace) + return + # At this point we know we need to launch a new run for the retry + strategy = get_reexecution_strategy(failed_run, instance) or DEFAULT_REEXECUTION_POLICY + tags = {RETRY_NUMBER_TAG: str(len(run_group_list))} new_run = instance.create_reexecuted_run( parent_run=failed_run, code_location=code_location, @@ -162,6 +177,7 @@ def retry_run( extra_tags=tags, use_parent_run_tags=True, ) + instance.add_run_tags(failed_run.run_id, {AUTO_RETRY_RUN_ID_TAG: new_run.run_id}) instance.report_engine_event( "Retrying the run", @@ -177,7 +193,6 @@ def retry_run( ) instance.submit_run(new_run.run_id, workspace) - instance.add_run_tags(failed_run.run_id, {AUTO_RETRY_RUN_ID_TAG: new_run.run_id}) def consume_new_runs_for_automatic_reexecution( @@ -190,15 +205,13 @@ def consume_new_runs_for_automatic_reexecution( it won't create another. The only exception is if the new run gets deleted, in which case we'd retry the run again. """ - for run, retry_number in filter_runs_to_should_retry( + for run in filter_runs_to_should_retry( [cast(DagsterRun, run_record.dagster_run) for run_record in run_records], workspace_process_context.instance, - workspace_process_context.instance.run_retries_max_retries, ): yield - try: - retry_run(run, retry_number, workspace_process_context) + retry_run(run, workspace_process_context) except Exception: error_info = DaemonErrorCapture.on_exception(exc_info=sys.exc_info()) workspace_process_context.instance.report_engine_event( @@ -206,3 +219,12 @@ def consume_new_runs_for_automatic_reexecution( run, engine_event_data=EngineEventData(error=error_info), ) + # Since something failed when retrying this run, mark that we will not retry it so that we + # don't retry it again in the future, and so that the tags reflect the state of the system. + # We may want to split out the kinds of exceptions and handle them differently in the future so + # that this can be more resiliant to transient errors. However, this would also require some changes + # to the EventLogConsumerDaemon so that the cursors are not updated in a way that prevents this run + # from being processed in the next tick. + workspace_process_context.instance.add_run_tags( + run_id=run.run_id, new_tags={WILL_RETRY_TAG: "false"} + ) diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_decorators.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_decorators.py index 21e9a37b2b2db..94c29a5265752 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_decorators.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_decorators.py @@ -1507,6 +1507,8 @@ def test_graph_inputs_error(): def _(): ... except DagsterInvalidDefinitionError as err: + assert "'_' decorated function does not have argument(s) 'start'" in str(err) + # Ensure that dagster type code path doesn't throw since we're using Nothing type. assert "except for Ins that have the Nothing dagster_type" not in str(err) try: @@ -1515,4 +1517,6 @@ def _(): ... def _(): ... except DagsterInvalidDefinitionError as err: + assert "'_' decorated function does not have argument(s) 'start'" in str(err) + # Ensure that dagster type code path doesn't throw since we're using Nothing type. assert "except for Ins that have the Nothing dagster_type" not in str(err) diff --git a/python_modules/libraries/dagster-airbyte/dagster_airbyte/resources.py b/python_modules/libraries/dagster-airbyte/dagster_airbyte/resources.py index 8e3c54fd9475a..0625b7c1073b0 100644 --- a/python_modules/libraries/dagster-airbyte/dagster_airbyte/resources.py +++ b/python_modules/libraries/dagster-airbyte/dagster_airbyte/resources.py @@ -1088,6 +1088,7 @@ def load_airbyte_cloud_asset_specs( Loading the asset specs for a given Airbyte Cloud workspace: .. code-block:: python + from dagster_airbyte import AirbyteCloudWorkspace, load_airbyte_cloud_asset_specs import dagster as dg diff --git a/python_modules/libraries/dagster-aws/dagster_aws/ecs/launcher.py b/python_modules/libraries/dagster-aws/dagster_aws/ecs/launcher.py index d5990e82c8bef..11ad43735a8ee 100644 --- a/python_modules/libraries/dagster-aws/dagster_aws/ecs/launcher.py +++ b/python_modules/libraries/dagster-aws/dagster_aws/ecs/launcher.py @@ -19,6 +19,7 @@ StringSource, _check as check, ) +from dagster._core.errors import DagsterInvariantViolationError from dagster._core.events import EngineEventData from dagster._core.instance import T_DagsterInstance from dagster._core.launcher.base import ( @@ -487,8 +488,6 @@ def launch_run(self, context: LaunchRunContext) -> None: ) command = self._get_command_args(args, context) image = self._get_image_for_run(context) - if image is None: - raise ValueError("Could not determine image for run") run_task_kwargs = self._run_task_kwargs(run, image, container_context) @@ -649,7 +648,7 @@ def _get_container_name(self, container_context: EcsContainerContext) -> str: return container_context.container_name or self.container_name def _run_task_kwargs( - self, run: DagsterRun, image: str, container_context: EcsContainerContext + self, run: DagsterRun, image: Optional[str], container_context: EcsContainerContext ) -> Dict[str, Any]: """Return a dictionary of args to launch the ECS task, registering a new task definition if needed. @@ -661,7 +660,7 @@ def _run_task_kwargs( if container_context.task_definition_arn: task_definition = container_context.task_definition_arn - else: + elif image is not None: family = self._get_run_task_definition_family(run) if self.task_definition_dict or not self.use_current_ecs_task_config: @@ -755,6 +754,11 @@ def _run_task_kwargs( ) task_definition = family + else: + # since image was not set, we cannot construct a task definition automatically + raise DagsterInvariantViolationError( + "Could not determine image to use for the run. It has to be provided in the code location: https://docs.dagster.io/concepts/code-locations/workspace-files#specifying-a-docker-image" + ) if self.use_current_ecs_task_config: current_task_metadata = get_current_ecs_task_metadata() diff --git a/python_modules/libraries/dagster-aws/dagster_aws/ecs/tasks.py b/python_modules/libraries/dagster-aws/dagster_aws/ecs/tasks.py index f8a86624e20af..1fc23421417cf 100644 --- a/python_modules/libraries/dagster-aws/dagster_aws/ecs/tasks.py +++ b/python_modules/libraries/dagster-aws/dagster_aws/ecs/tasks.py @@ -204,7 +204,7 @@ def get_task_definition_dict_from_current_task( ecs, family, current_task, - image, + image: str, container_name, environment, command=None, diff --git a/python_modules/libraries/dagster-aws/dagster_aws/pipes/clients/ecs.py b/python_modules/libraries/dagster-aws/dagster_aws/pipes/clients/ecs.py index cd3ac389a2fca..c9830e586c0a2 100644 --- a/python_modules/libraries/dagster-aws/dagster_aws/pipes/clients/ecs.py +++ b/python_modules/libraries/dagster-aws/dagster_aws/pipes/clients/ecs.py @@ -1,5 +1,5 @@ from pprint import pformat -from typing import TYPE_CHECKING, Any, Dict, Optional, Union, cast +from typing import TYPE_CHECKING, Any, Dict, Optional, TypedDict, Union, cast import boto3 import botocore @@ -17,6 +17,7 @@ PipesMessageReader, ) from dagster._core.pipes.utils import PipesEnvContextInjector, open_pipes_session +from typing_extensions import NotRequired from dagster_aws.pipes.message_readers import PipesCloudWatchLogReader, PipesCloudWatchMessageReader @@ -29,6 +30,20 @@ ) +class WaiterConfig(TypedDict): + """A WaiterConfig representing the configuration of the waiter. + + Args: + Delay (NotRequired[int]): The amount of time in seconds to wait between attempts. Defaults to 6. + MaxAttempts (NotRequired[int]): The maximum number of attempts to be made. Defaults to 1000000 + By default the waiter is configured to wait up to 70 days (waiter_delay*waiter_max_attempts). + See `Boto3 API Documentation `_ + """ + + Delay: NotRequired[int] + MaxAttempts: NotRequired[int] + + @experimental class PipesECSClient(PipesClient, TreatAsResourceParam): """A pipes client for running AWS ECS tasks. @@ -66,6 +81,7 @@ def run( run_task_params: "RunTaskRequestRequestTypeDef", extras: Optional[Dict[str, Any]] = None, pipes_container_name: Optional[str] = None, + waiter_config: Optional[WaiterConfig] = None, ) -> PipesClientCompletedInvocation: """Run ECS tasks, enriched with the pipes protocol. @@ -77,11 +93,14 @@ def run( 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, and using :py:class:`PipesCloudWatchMessageReader`, specify the container name which will be running Pipes. + waiter_config (Optional[WaiterConfig]): Optional waiter configuration to use. Defaults to 70 days (Delay: 6, MaxAttempts: 1000000). Returns: PipesClientCompletedInvocation: Wrapper containing results reported by the external process. """ + waiter_config = waiter_config or WaiterConfig(Delay=6, MaxAttempts=1000000) + with open_pipes_session( context=context, message_reader=self._message_reader, @@ -240,7 +259,9 @@ def get_cloudwatch_params(container_name: str) -> Optional[Dict[str, str]]: ), ) - response = self._wait_for_completion(response, cluster=cluster) + response = self._wait_for_completion( + response, cluster=cluster, waiter_config=waiter_config + ) # check for failed containers failed_containers = {} @@ -269,7 +290,10 @@ def get_cloudwatch_params(container_name: str) -> Optional[Dict[str, str]]: ) def _wait_for_completion( - self, start_response: "RunTaskResponseTypeDef", cluster: Optional[str] = None + self, + start_response: "RunTaskResponseTypeDef", + cluster: Optional[str] = None, + waiter_config: Optional[WaiterConfig] = None, ) -> "DescribeTasksResponseTypeDef": waiter = self._client.get_waiter("tasks_stopped") @@ -278,7 +302,10 @@ def _wait_for_completion( if cluster: params["cluster"] = cluster - waiter.wait(**params) + waiter_params = {"WaiterConfig": waiter_config, **params} if waiter_config else params + + waiter.wait(**waiter_params) + return self._client.describe_tasks(**params) def _extract_dagster_metadata( diff --git a/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/fake_ecs.py b/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/fake_ecs.py index 6df91b38860c3..156d3f64ddbf2 100644 --- a/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/fake_ecs.py +++ b/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/fake_ecs.py @@ -7,6 +7,7 @@ from typing import Dict, List, Optional, cast import boto3 +import botocore @dataclass @@ -249,12 +250,27 @@ def __init__(self, client: LocalECSMockClient, waiter_name: str): self.waiter_name = waiter_name def wait(self, **kwargs): + waiter_config = kwargs.pop("WaiterConfig", {"MaxAttempts": 100, "Delay": 6}) + max_attempts = int(waiter_config.get("MaxAttempts", 100)) + delay = int(waiter_config.get("Delay", 6)) + num_attempts = 0 + if self.waiter_name == "tasks_stopped": while True: response = self.client.describe_tasks(**kwargs) + num_attempts += 1 + if all(task["lastStatus"] == "STOPPED" for task in response["tasks"]): return - time.sleep(0.1) + + if num_attempts >= max_attempts: + raise botocore.exceptions.WaiterError( + name=self.waiter_name, + reason="Max attempts exceeded", + last_response=response, + ) + + time.sleep(delay) else: raise NotImplementedError(f"Waiter {self.waiter_name} is not implemented") diff --git a/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/test_pipes.py b/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/test_pipes.py index d1fb44a3a058c..f8a25cf85dd18 100644 --- a/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/test_pipes.py +++ b/python_modules/libraries/dagster-aws/dagster_aws_tests/pipes_tests/test_pipes.py @@ -17,6 +17,7 @@ from uuid import uuid4 import boto3 +import botocore import pytest from dagster import AssetsDefinition, asset, materialize, open_pipes_session from dagster._core.definitions.asset_check_spec import AssetCheckKey, AssetCheckSpec @@ -46,6 +47,7 @@ PipesS3LogReader, PipesS3MessageReader, ) +from dagster_aws.pipes.clients.ecs import WaiterConfig from dagster_aws_tests.pipes_tests.fake_ecs import LocalECSMockClient from dagster_aws_tests.pipes_tests.fake_glue import LocalGlueMockClient from dagster_aws_tests.pipes_tests.fake_lambda import ( @@ -862,6 +864,10 @@ def ecs_asset(context: AssetExecutionContext, pipes_ecs_client: PipesECSClient): ] }, }, + waiter_config=WaiterConfig( + Delay=int(os.getenv("WAIT_DELAY", "6")), + MaxAttempts=int(os.getenv("WAIT_MAX_ATTEMPTS", "1000000")), + ), ).get_results() @@ -935,6 +941,37 @@ def materialize_asset(env, return_dict): assert return_dict[0]["tasks"][0]["stoppedReason"] == "Dagster process was interrupted" +def test_ecs_pipes_waiter_config(pipes_ecs_client: PipesECSClient): + with instance_for_test() as instance: + """ + Test Error is thrown when the wait delay is less than the processing time. + """ + os.environ.update({"WAIT_DELAY": "1", "WAIT_MAX_ATTEMPTS": "1", "SLEEP_SECONDS": "2"}) + with pytest.raises(botocore.exceptions.WaiterError, match=r".* Max attempts exceeded"): # pyright: ignore (reportAttributeAccessIssue) + materialize( + [ecs_asset], instance=instance, resources={"pipes_ecs_client": pipes_ecs_client} + ) + + """ + Test Error is thrown when the wait attempts * wait delay is less than the processing time. + """ + os.environ.update({"WAIT_DELAY": "1", "WAIT_MAX_ATTEMPTS": "2", "SLEEP_SECONDS": "3"}) + with pytest.raises(botocore.exceptions.WaiterError, match=r".* Max attempts exceeded"): # pyright: ignore (reportAttributeAccessIssue) + materialize( + [ecs_asset], instance=instance, resources={"pipes_ecs_client": pipes_ecs_client} + ) + + """ + Test asset is materialized successfully when the wait attempts * wait delay is greater than the processing time. + """ + os.environ.update({"WAIT_DELAY": "1", "WAIT_MAX_ATTEMPTS": "10", "SLEEP_SECONDS": "2"}) + materialize( + [ecs_asset], instance=instance, resources={"pipes_ecs_client": pipes_ecs_client} + ) + mat = instance.get_latest_materialization_event(ecs_asset.key) + assert mat and mat.asset_materialization + + EMR_SERVERLESS_APP_NAME = "Example" diff --git a/python_modules/libraries/dagster-components/README.md b/python_modules/libraries/dagster-components/README.md index 885fc94f46adf..ef23ccff2e4d2 100644 --- a/python_modules/libraries/dagster-components/README.md +++ b/python_modules/libraries/dagster-components/README.md @@ -1,4 +1,3 @@ # dagster-components -Experimental API for defining Dagster definitions factories ("components"). -Includes the `dg` CLI tool. +Experimental Python API for defining Dagster definitions factories ("components"). diff --git a/python_modules/libraries/dagster-components/dagster_components/__init__.py b/python_modules/libraries/dagster-components/dagster_components/__init__.py index 0f4d31450868a..f5463bf0d60f8 100644 --- a/python_modules/libraries/dagster-components/dagster_components/__init__.py +++ b/python_modules/libraries/dagster-components/dagster_components/__init__.py @@ -1,19 +1,14 @@ +from dagster._core.libraries import DagsterLibraryRegistry + from dagster_components.core.component import ( Component as Component, ComponentLoadContext as ComponentLoadContext, ComponentRegistry as ComponentRegistry, + component as component, ) from dagster_components.core.component_defs_builder import ( build_defs_from_toplevel_components_folder as build_defs_from_toplevel_components_folder, ) -from dagster_components.impls.dbt_project import DbtProjectComponent -from dagster_components.impls.pipes_subprocess_script_collection import ( - PipesSubprocessScriptCollection, -) -from dagster_components.impls.sling_replication import SlingReplicationComponent +from dagster_components.version import __version__ as __version__ -__component_registry__ = { - "pipes_subprocess_script_collection": PipesSubprocessScriptCollection, - "sling_replication": SlingReplicationComponent, - "dbt_project": DbtProjectComponent, -} +DagsterLibraryRegistry.register("dagster-components", __version__) diff --git a/python_modules/libraries/dagster-components/dagster_components/cli/__init__.py b/python_modules/libraries/dagster-components/dagster_components/cli/__init__.py index 5b179e78098cb..025e67200f49c 100644 --- a/python_modules/libraries/dagster-components/dagster_components/cli/__init__.py +++ b/python_modules/libraries/dagster-components/dagster_components/cli/__init__.py @@ -2,11 +2,13 @@ from dagster.version import __version__ from dagster_components.cli.generate import generate_cli +from dagster_components.cli.list import list_cli def create_dagster_components_cli(): commands = { "generate": generate_cli, + "list": list_cli, } @click.group( diff --git a/python_modules/libraries/dagster-components/dagster_components/cli/generate.py b/python_modules/libraries/dagster-components/dagster_components/cli/generate.py index 60aece54431a6..438d8ccb3b578 100644 --- a/python_modules/libraries/dagster-components/dagster_components/cli/generate.py +++ b/python_modules/libraries/dagster-components/dagster_components/cli/generate.py @@ -1,24 +1,16 @@ -import os import sys from pathlib import Path -from typing import Optional +from typing import Optional, Tuple import click from pydantic import TypeAdapter -from dagster_components import ComponentRegistry, __component_registry__ +from dagster_components import ComponentRegistry from dagster_components.core.deployment import ( CodeLocationProjectContext, - DeploymentProjectContext, is_inside_code_location_project, - is_inside_deployment_project, -) -from dagster_components.generate import ( - generate_code_location, - generate_component_instance, - generate_component_type, - generate_deployment, ) +from dagster_components.generate import generate_component_instance @click.group(name="generate") @@ -26,68 +18,18 @@ def generate_cli() -> None: """Commands for generating Dagster components and related entities.""" -@generate_cli.command(name="deployment") -@click.argument("path", type=str) -def generate_deployment_command(path: str) -> None: - """Generate a Dagster deployment instance.""" - dir_abspath = os.path.abspath(path) - if os.path.exists(dir_abspath): - click.echo( - click.style(f"A file or directory at {dir_abspath} already exists. ", fg="red") - + "\nPlease delete the contents of this path or choose another location." - ) - sys.exit(1) - generate_deployment(path) - - -@generate_cli.command(name="code-location") -@click.argument("name", type=str) -def generate_code_location_command(name: str) -> None: - """Generate a Dagster code location inside a component.""" - if not is_inside_deployment_project(Path(".")): - click.echo( - click.style("This command must be run inside a Dagster deployment project.", fg="red") - ) - sys.exit(1) - - context = DeploymentProjectContext.from_path(Path.cwd()) - if context.has_code_location(name): - click.echo(click.style(f"A code location named {name} already exists.", fg="red")) - sys.exit(1) - - code_location_path = os.path.join(context.code_location_root_path, name) - generate_code_location(code_location_path) - - -@generate_cli.command(name="component-type") -@click.argument("name", type=str) -def generate_component_type_command(name: str) -> None: - """Generate a Dagster component instance.""" - if not is_inside_code_location_project(Path(".")): - click.echo( - click.style( - "This command must be run inside a Dagster code location project.", fg="red" - ) - ) - sys.exit(1) - - context = CodeLocationProjectContext.from_path( - Path.cwd(), ComponentRegistry(__component_registry__) - ) - if context.has_component_type(name): - click.echo(click.style(f"A component type named `{name}` already exists.", fg="red")) - sys.exit(1) - - generate_component_type(context.component_types_root_path, name) - - @generate_cli.command(name="component") -@click.argument("component-type", type=str) -@click.argument("name", type=str) -@click.option("--params", type=str, default=None) -def generate_component_command(component_type: str, name: str, params: Optional[str]) -> None: - """Generate a Dagster component instance.""" - if not is_inside_code_location_project(Path(".")): +@click.argument("component_type", type=str) +@click.argument("component_name", type=str) +@click.option("--json-params", type=str, default=None) +@click.argument("extra_args", nargs=-1, type=str) +def generate_component_command( + component_type: str, + component_name: str, + json_params: Optional[str], + extra_args: Tuple[str, ...], +) -> None: + if not is_inside_code_location_project(Path.cwd()): click.echo( click.style( "This command must be run inside a Dagster code location project.", fg="red" @@ -96,23 +38,32 @@ def generate_component_command(component_type: str, name: str, params: Optional[ sys.exit(1) context = CodeLocationProjectContext.from_path( - Path.cwd(), ComponentRegistry(__component_registry__) + Path.cwd(), ComponentRegistry.from_entry_point_discovery() ) if not context.has_component_type(component_type): click.echo( click.style(f"No component type `{component_type}` could be resolved.", fg="red") ) sys.exit(1) - elif context.has_component_instance(name): - click.echo(click.style(f"A component instance named `{name}` already exists.", fg="red")) - sys.exit(1) component_type_cls = context.get_component_type(component_type) - generate_params = ( - TypeAdapter(component_type_cls.generate_params_schema).validate_json(params) - if params - else None - ) + generate_params_schema = component_type_cls.generate_params_schema + generate_params_cli = getattr(generate_params_schema, "cli", None) + if generate_params_schema is None: + generate_params = None + elif json_params is not None: + generate_params = TypeAdapter(generate_params_schema).validate_json(json_params) + elif generate_params_cli is not None: + inner_ctx = click.Context(generate_params_cli) + generate_params_cli.parse_args(inner_ctx, list(extra_args)) + generate_params = inner_ctx.invoke(generate_params_schema.cli, **inner_ctx.params) + else: + generate_params = None + generate_component_instance( - context.component_instances_root_path, name, component_type_cls, generate_params + context.component_instances_root_path, + component_name, + component_type_cls, + component_type, + generate_params, ) diff --git a/python_modules/libraries/dagster-components/dagster_components/cli/list.py b/python_modules/libraries/dagster-components/dagster_components/cli/list.py new file mode 100644 index 0000000000000..9368a9fa1206a --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components/cli/list.py @@ -0,0 +1,40 @@ +import json +import sys +from pathlib import Path +from typing import Any, Dict + +import click + +from dagster_components.core.component import ComponentRegistry +from dagster_components.core.deployment import ( + CodeLocationProjectContext, + is_inside_code_location_project, +) + + +@click.group(name="generate") +def list_cli(): + """Commands for listing Dagster components and related entities.""" + + +@list_cli.command(name="component-types") +def list_component_types_command() -> None: + """List registered Dagster components.""" + if not is_inside_code_location_project(Path.cwd()): + click.echo( + click.style( + "This command must be run inside a Dagster code location project.", fg="red" + ) + ) + sys.exit(1) + + context = CodeLocationProjectContext.from_path( + Path.cwd(), ComponentRegistry.from_entry_point_discovery() + ) + output: Dict[str, Any] = {} + for component_type in context.list_component_types(): + # package, name = component_type.rsplit(".", 1) + output[component_type] = { + "name": component_type, + } + click.echo(json.dumps(output)) diff --git a/python_modules/libraries/dagster-components/dagster_components/core/component.py b/python_modules/libraries/dagster-components/dagster_components/core/component.py index e0bf155204054..fe15ed559e48e 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/component.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/component.py @@ -1,8 +1,12 @@ import copy +import importlib +import importlib.metadata +import sys from abc import ABC, abstractmethod from types import ModuleType -from typing import TYPE_CHECKING, Any, ClassVar, Dict, Iterable, Mapping, Optional, Type +from typing import TYPE_CHECKING, Any, ClassVar, Dict, Iterable, Mapping, Optional, Sequence, Type +from dagster import _check as check from dagster._core.errors import DagsterError from dagster._utils import snakecase from typing_extensions import Self @@ -16,12 +20,8 @@ class ComponentDeclNode: ... class Component(ABC): name: ClassVar[Optional[str]] = None - defs_params_schema: ClassVar[Type] = Type[None] - generate_params_schema: ClassVar[Type] = Type[None] - - @classmethod - def registered_name(cls) -> str: - return cls.name or snakecase(cls.__name__) + component_params_schema: ClassVar = None + generate_params_schema: ClassVar = None @classmethod def generate_files(cls, params: Any) -> Optional[Mapping[str, Any]]: ... @@ -36,7 +36,33 @@ def from_decl_node( ) -> Self: ... +def get_entry_points_from_python_environment(group: str) -> Sequence[importlib.metadata.EntryPoint]: + if sys.version_info >= (3, 10): + return importlib.metadata.entry_points(group=group) + else: + return importlib.metadata.entry_points().get(group, []) + + +COMPONENTS_ENTRY_POINT_GROUP = "dagster.components" + + class ComponentRegistry: + @classmethod + def from_entry_point_discovery(cls) -> "ComponentRegistry": + components: Dict[str, Type[Component]] = {} + for entry_point in get_entry_points_from_python_environment(COMPONENTS_ENTRY_POINT_GROUP): + root_module = entry_point.load() + if not isinstance(root_module, ModuleType): + raise DagsterError( + f"Invalid entry point {entry_point.name} in group {COMPONENTS_ENTRY_POINT_GROUP}. " + f"Value expected to be a module, got {root_module}." + ) + for component in get_registered_components_in_module(root_module): + key = f"{entry_point.name}.{get_component_name(component)}" + components[key] = component + + return cls(components) + def __init__(self, components: Dict[str, Type[Component]]): self._components: Dict[str, Type[Component]] = copy.copy(components) @@ -62,7 +88,7 @@ def __repr__(self) -> str: return f"" -def register_components_in_module(registry: ComponentRegistry, root_module: ModuleType) -> None: +def get_registered_components_in_module(root_module: ModuleType) -> Iterable[Type[Component]]: from dagster._core.definitions.load_assets_from_modules import ( find_modules_in_package, find_subclasses_in_module, @@ -70,9 +96,8 @@ def register_components_in_module(registry: ComponentRegistry, root_module: Modu for module in find_modules_in_package(root_module): for component in find_subclasses_in_module(module, (Component,)): - if component is Component: - continue - registry.register(component.registered_name(), component) + if is_registered_component(component): + yield component class ComponentLoadContext: @@ -89,3 +114,43 @@ def for_test( return ComponentLoadContext( resources=resources or {}, registry=registry or ComponentRegistry.empty() ) + + +COMPONENT_REGISTRY_KEY_ATTR = "__dagster_component_registry_key" + + +def component( + cls: Optional[Type[Component]] = None, + *, + name: Optional[str] = None, +) -> Any: + if cls is None: + + def wrapper(actual_cls: Type[Component]) -> Type[Component]: + check.inst_param(actual_cls, "actual_cls", type) + setattr( + actual_cls, + COMPONENT_REGISTRY_KEY_ATTR, + name or snakecase(actual_cls.__name__), + ) + return actual_cls + + return wrapper + else: + # called without params + check.inst_param(cls, "cls", type) + setattr(cls, COMPONENT_REGISTRY_KEY_ATTR, name or snakecase(cls.__name__)) + return cls + + +def is_registered_component(cls: Type) -> bool: + return hasattr(cls, COMPONENT_REGISTRY_KEY_ATTR) + + +def get_component_name(component_type: Type[Component]) -> str: + check.param_invariant( + is_registered_component(component_type), + "component_type", + "Expected a registered component. Use @component to register a component.", + ) + return getattr(component_type, COMPONENT_REGISTRY_KEY_ATTR) diff --git a/python_modules/libraries/dagster-components/dagster_components/core/component_decl_builder.py b/python_modules/libraries/dagster-components/dagster_components/core/component_decl_builder.py index 82e94c6e2d340..160ad0b8ef93a 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/component_decl_builder.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/component_decl_builder.py @@ -8,15 +8,15 @@ from dagster_components.core.component import ComponentDeclNode -class DefsFileModel(BaseModel): - component_type: str - component_params: Optional[Mapping[str, Any]] = None +class ComponentFileModel(BaseModel): + type: str + params: Optional[Mapping[str, Any]] = None @record class YamlComponentDecl(ComponentDeclNode): path: Path - defs_file_model: DefsFileModel + component_file_model: ComponentFileModel @record @@ -27,19 +27,19 @@ class ComponentFolder(ComponentDeclNode): def path_to_decl_node(path: Path) -> Optional[ComponentDeclNode]: # right now, we only support two types of components, both of which are folders - # if the folder contains a defs.yml file, it's a component instance + # if the folder contains a component.yaml file, it's a component instance # otherwise, it's a folder containing sub-components if not path.is_dir(): return None - defs_path = path / "defs.yml" + component_path = path / "component.yaml" - if defs_path.exists(): - defs_file_model = parse_yaml_file_to_pydantic( - DefsFileModel, defs_path.read_text(), str(path) + if component_path.exists(): + component_file_model = parse_yaml_file_to_pydantic( + ComponentFileModel, component_path.read_text(), str(path) ) - return YamlComponentDecl(path=path, defs_file_model=defs_file_model) + return YamlComponentDecl(path=path, component_file_model=component_file_model) subs = [] for subpath in path.iterdir(): diff --git a/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py b/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py index 653bb65597f3e..3d3e7ab4d7851 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py @@ -1,5 +1,9 @@ +import importlib +import importlib.util +import inspect from pathlib import Path -from typing import TYPE_CHECKING, List, Mapping, Optional, Sequence +from types import ModuleType +from typing import TYPE_CHECKING, List, Mapping, Optional, Sequence, Type from dagster._utils.warnings import suppress_dagster_warnings @@ -8,6 +12,8 @@ ComponentDeclNode, ComponentLoadContext, ComponentRegistry, + get_component_name, + is_registered_component, ) from dagster_components.core.component_decl_builder import ( ComponentFolder, @@ -20,12 +26,24 @@ from dagster._core.definitions.definitions_class import Definitions +def load_module_from_path(module_name, path) -> ModuleType: + # Create a spec from the file path + spec = importlib.util.spec_from_file_location(module_name, path) + if spec is None: + raise ImportError(f"Cannot create a module spec from path: {path}") + + # Create and load the module + module = importlib.util.module_from_spec(spec) + assert spec.loader, "Must have a loader" + spec.loader.exec_module(module) + return module + + def build_components_from_decl_node( context: ComponentLoadContext, decl_node: ComponentDeclNode ) -> Sequence[Component]: if isinstance(decl_node, YamlComponentDecl): - parsed_defs = decl_node.defs_file_model - component_type = context.registry.get(parsed_defs.component_type) + component_type = component_type_from_yaml_decl(context, decl_node) return [component_type.from_decl_node(context, decl_node)] elif isinstance(decl_node, ComponentFolder): components = [] @@ -36,6 +54,34 @@ def build_components_from_decl_node( raise NotImplementedError(f"Unknown component type {decl_node}") +def component_type_from_yaml_decl( + context: ComponentLoadContext, decl_node: YamlComponentDecl +) -> Type: + parsed_defs = decl_node.component_file_model + if parsed_defs.type.startswith("."): + component_registry_key = parsed_defs.type[1:] + + # Iterate over Python files in the folder + for py_file in decl_node.path.glob("*.py"): + module_name = py_file.stem + + module = load_module_from_path(module_name, decl_node.path / f"{module_name}.py") + + for _name, obj in inspect.getmembers(module, inspect.isclass): + assert isinstance(obj, Type) + if ( + is_registered_component(obj) + and get_component_name(obj) == component_registry_key + ): + return obj + + raise Exception( + f"Could not find component type {component_registry_key} in {decl_node.path}" + ) + + return context.registry.get(parsed_defs.type) + + def build_components_from_component_folder( context: ComponentLoadContext, path: Path, @@ -84,7 +130,9 @@ def build_defs_from_toplevel_components_folder( """Build a Definitions object from an entire component hierarchy.""" from dagster._core.definitions.definitions_class import Definitions - context = CodeLocationProjectContext.from_path(path, registry or ComponentRegistry.empty()) + context = CodeLocationProjectContext.from_path( + path, registry or ComponentRegistry.from_entry_point_discovery() + ) all_defs: List[Definitions] = [] for component in context.component_instances: diff --git a/python_modules/libraries/dagster-components/dagster_components/core/deployment.py b/python_modules/libraries/dagster-components/dagster_components/core/deployment.py index 64a60cc278188..ca52531dc4e77 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/deployment.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/deployment.py @@ -1,34 +1,16 @@ -import importlib.util import os -import sys from pathlib import Path from typing import Final, Iterable, Type +import tomli from dagster._core.errors import DagsterError from typing_extensions import Self -from dagster_components.core.component import ( - Component, - ComponentRegistry, - register_components_in_module, -) +from dagster_components.core.component import Component, ComponentRegistry - -def is_inside_deployment_project(path: Path) -> bool: - try: - _resolve_deployment_root_path(path) - return True - except DagsterError: - return False - - -def _resolve_deployment_root_path(path: Path) -> str: - current_path = os.path.abspath(path) - while not _is_deployment_root(current_path): - current_path = os.path.dirname(current_path) - if current_path == "/": - raise DagsterError("Cannot find deployment root") - return current_path +# Code location +_CODE_LOCATION_CUSTOM_COMPONENTS_DIR: Final = "lib" +_CODE_LOCATION_COMPONENT_INSTANCES_DIR: Final = "components" def is_inside_code_location_project(path: Path) -> bool: @@ -39,89 +21,43 @@ def is_inside_code_location_project(path: Path) -> bool: return False -def _resolve_code_location_root_path(path: Path) -> str: - current_path = os.path.abspath(path) +def _resolve_code_location_root_path(path: Path) -> Path: + current_path = path.absolute() while not _is_code_location_root(current_path): - current_path = os.path.dirname(current_path) - if current_path == "/": + current_path = current_path.parent + if str(current_path) == "/": raise DagsterError("Cannot find code location root") return current_path -def _is_deployment_root(path: str) -> bool: - return os.path.exists(os.path.join(path, "code_locations")) - - -def _is_code_location_root(path: str) -> bool: - return os.path.basename(os.path.dirname(path)) == "code_locations" - - -# Deployment -_DEPLOYMENT_CODE_LOCATIONS_DIR: Final = "code_locations" - -# Code location -_CODE_LOCATION_CUSTOM_COMPONENTS_DIR: Final = "lib" -_CODE_LOCATION_COMPONENT_INSTANCES_DIR: Final = "components" - - -class DeploymentProjectContext: - @classmethod - def from_path(cls, path: Path) -> Self: - return cls(root_path=_resolve_deployment_root_path(path)) - - def __init__(self, root_path: str): - self._root_path = root_path - - @property - def deployment_root(self) -> str: - return self._root_path - - @property - def code_location_root_path(self) -> str: - return os.path.join(self._root_path, _DEPLOYMENT_CODE_LOCATIONS_DIR) - - def has_code_location(self, name: str) -> bool: - return os.path.exists(os.path.join(self._root_path, "code_locations", name)) +def _is_code_location_root(path: Path) -> bool: + if (path / "pyproject.toml").exists(): + with open(path / "pyproject.toml") as f: + toml = tomli.loads(f.read()) + return bool(toml.get("tool", {}).get("dagster")) + return False class CodeLocationProjectContext: @classmethod def from_path(cls, path: Path, component_registry: "ComponentRegistry") -> Self: root_path = _resolve_code_location_root_path(path) - name = os.path.basename(root_path) - - # TODO: Rm when a more robust solution is implemented - # Make sure we can import from the cwd - if sys.path[0] != "": - sys.path.insert(0, "") - - components_lib_module = f"{name}.{_CODE_LOCATION_CUSTOM_COMPONENTS_DIR}" - module = importlib.import_module(components_lib_module) - register_components_in_module(component_registry, module) - return cls( - deployment_context=DeploymentProjectContext.from_path(path), - root_path=root_path, + root_path=str(root_path), name=os.path.basename(root_path), component_registry=component_registry, ) def __init__( self, - deployment_context: DeploymentProjectContext, root_path: str, name: str, component_registry: "ComponentRegistry", ): - self._deployment_context = deployment_context self._root_path = root_path self._name = name self._component_registry = component_registry - @property - def deployment_context(self) -> DeploymentProjectContext: - return self._deployment_context - @property def component_types_root_path(self) -> str: return os.path.join(self._root_path, self._name, _CODE_LOCATION_CUSTOM_COMPONENTS_DIR) @@ -142,6 +78,9 @@ def get_component_type(self, name: str) -> Type[Component]: raise DagsterError(f"No component type named {name}") return self._component_registry.get(name) + def list_component_types(self) -> Iterable[str]: + return sorted(self._component_registry.keys()) + def get_component_instance_path(self, name: str) -> str: if name not in self.component_instances: raise DagsterError(f"No component instance named {name}") @@ -153,8 +92,10 @@ def component_instances_root_path(self) -> str: @property def component_instances(self) -> Iterable[str]: - return os.listdir( - os.path.join(self._root_path, self._name, _CODE_LOCATION_COMPONENT_INSTANCES_DIR) + return sorted( + os.listdir( + os.path.join(self._root_path, self._name, _CODE_LOCATION_COMPONENT_INSTANCES_DIR) + ) ) def has_component_instance(self, name: str) -> bool: diff --git a/python_modules/libraries/dagster-components/dagster_components/core/dsl_schema.py b/python_modules/libraries/dagster-components/dagster_components/core/dsl_schema.py new file mode 100644 index 0000000000000..bf63f7aba6e76 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components/core/dsl_schema.py @@ -0,0 +1,8 @@ +from typing import Dict, Optional + +from pydantic import BaseModel + + +class OpSpecBaseModel(BaseModel): + name: Optional[str] = None + tags: Optional[Dict[str, str]] = None diff --git a/python_modules/libraries/dagster-components/dagster_components/generate.py b/python_modules/libraries/dagster-components/dagster_components/generate.py index 2734ee9320af1..dcda0cd58967b 100644 --- a/python_modules/libraries/dagster-components/dagster_components/generate.py +++ b/python_modules/libraries/dagster-components/dagster_components/generate.py @@ -1,15 +1,16 @@ import os +from pathlib import Path from typing import Any, Type import click import yaml from dagster._generate.generate import generate_project -from dagster._utils import camelcase, pushd +from dagster._utils import pushd from dagster_components.core.component import Component -class DefsDumper(yaml.Dumper): +class ComponentDumper(yaml.Dumper): def write_line_break(self) -> None: # add an extra line break between top-level keys if self.indent == 0: @@ -17,44 +18,12 @@ def write_line_break(self) -> None: super().write_line_break() -def generate_deployment(path: str) -> None: - click.echo(f"Creating a Dagster deployment at {path}.") - - generate_project( - path=path, - name_placeholder="DEPLOYMENT_NAME_PLACEHOLDER", - templates_path=os.path.join( - os.path.dirname(__file__), "templates", "DEPLOYMENT_NAME_PLACEHOLDER" - ), - ) - - -def generate_code_location(path: str) -> None: - click.echo(f"Creating a Dagster code location at {path}.") - - generate_project( - path=path, - name_placeholder="CODE_LOCATION_NAME_PLACEHOLDER", - templates_path=os.path.join( - os.path.dirname(__file__), "templates", "CODE_LOCATION_NAME_PLACEHOLDER" - ), - ) - - -def generate_component_type(root_path: str, name: str) -> None: - click.echo(f"Creating a Dagster component type at {root_path}/{name}.py.") - - generate_project( - path=root_path, - name_placeholder="COMPONENT_TYPE_NAME_PLACEHOLDER", - templates_path=os.path.join(os.path.dirname(__file__), "templates", "COMPONENT_TYPE"), - project_name=name, - component_type_class_name=camelcase(name), - ) - - def generate_component_instance( - root_path: str, name: str, component_type: Type[Component], generate_params: Any + root_path: str, + name: str, + component_type: Type[Component], + component_type_name: str, + generate_params: Any, ) -> None: click.echo(f"Creating a Dagster component instance at {root_path}/{name}.py.") @@ -66,16 +35,12 @@ def generate_component_instance( os.path.dirname(__file__), "templates", "COMPONENT_INSTANCE_NAME_PLACEHOLDER" ), project_name=name, - component_type=component_type.registered_name(), + component_type=component_type_name, ) with pushd(component_instance_root_path): - defs_data: dict = {"component_type": component_type.registered_name()} - component_params = ( - component_type.generate_files(generate_params) - if generate_params - else component_type.generate_files() # type: ignore + component_params = component_type.generate_files(generate_params) + component_data = {"type": component_type_name, "params": component_params or {}} + with open(Path(component_instance_root_path) / "component.yaml", "w") as f: + yaml.dump( + component_data, f, Dumper=ComponentDumper, sort_keys=False, default_flow_style=False ) - if component_params: - defs_data["component_params"] = component_params - with open("defs.yml", "w") as f: - yaml.dump(defs_data, f, Dumper=DefsDumper, sort_keys=False, default_flow_style=False) diff --git a/python_modules/libraries/dagster-components/dagster_components/impls/dbt_project.py b/python_modules/libraries/dagster-components/dagster_components/impls/dbt_project.py deleted file mode 100644 index be3ec4814d8e5..0000000000000 --- a/python_modules/libraries/dagster-components/dagster_components/impls/dbt_project.py +++ /dev/null @@ -1,72 +0,0 @@ -import os -from pathlib import Path -from typing import Any, Mapping, Optional - -import dagster._check as check -from dagster._core.definitions.definitions_class import Definitions -from dagster._utils import pushd -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets -from dagster_embedded_elt.sling.resources import AssetExecutionContext -from dbt.cli.main import dbtRunner -from pydantic import BaseModel, TypeAdapter -from typing_extensions import Self - -from dagster_components import Component, ComponentLoadContext -from dagster_components.core.component_decl_builder import ComponentDeclNode, YamlComponentDecl - - -class DbtProjectParams(BaseModel): - dbt: DbtCliResource - - -class DbtGenerateParams(BaseModel): - init: bool = False - project_path: Optional[str] = None - - -class DbtProjectComponent(Component): - params_schema = DbtProjectParams - generate_params_schema = DbtGenerateParams - - def __init__(self, dbt_resource: DbtCliResource): - self.dbt_resource = dbt_resource - - @classmethod - def registered_name(cls) -> str: - return "dbt_project" - - @classmethod - def from_decl_node(cls, context: ComponentLoadContext, decl_node: ComponentDeclNode) -> Self: - assert isinstance(decl_node, YamlComponentDecl) - - # all paths should be resolved relative to the directory we're in - with pushd(str(decl_node.path)): - loaded_params = TypeAdapter(cls.params_schema).validate_python( - decl_node.defs_file_model.component_params - ) - return cls(dbt_resource=loaded_params.dbt) - - def build_defs(self, context: ComponentLoadContext) -> Definitions: - project = DbtProject(self.dbt_resource.project_dir) - project.prepare_if_dev() - - @dbt_assets(manifest=project.manifest_path, project=project) - def _fn(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - return Definitions(assets=[_fn], resources={"dbt": self.dbt_resource}) - - @classmethod - def generate_files(cls, params: DbtGenerateParams) -> Mapping[str, Any]: - if params.project_path: - relative_path = os.path.relpath(params.project_path, start=os.getcwd()) - elif params.init: - dbtRunner().invoke(["init"]) - subpaths = list(Path(os.getcwd()).iterdir()) - check.invariant(len(subpaths) == 1, "Expected exactly one subpath to be created.") - # this path should be relative to this directory - relative_path = subpaths[0].name - else: - relative_path = None - - return {"dbt": {"project_dir": relative_path}} diff --git a/python_modules/libraries/dagster-components/dagster_components/impls/__init__.py b/python_modules/libraries/dagster-components/dagster_components/lib/__init__.py similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/impls/__init__.py rename to python_modules/libraries/dagster-components/dagster_components/lib/__init__.py diff --git a/python_modules/libraries/dagster-components/dagster_components/lib/dbt_project.py b/python_modules/libraries/dagster-components/dagster_components/lib/dbt_project.py new file mode 100644 index 0000000000000..84d53d337e0ac --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components/lib/dbt_project.py @@ -0,0 +1,137 @@ +import os +from pathlib import Path +from typing import Any, Mapping, Optional + +import click +import dagster._check as check +from dagster._core.definitions.asset_key import AssetKey +from dagster._core.definitions.definitions_class import Definitions +from dagster._utils import pushd +from dagster_dbt import DagsterDbtTranslator, DbtCliResource, DbtProject, dbt_assets +from dagster_embedded_elt.sling.resources import AssetExecutionContext +from dbt.cli.main import dbtRunner +from jinja2 import Template +from pydantic import BaseModel, Field, TypeAdapter +from typing_extensions import Self + +from dagster_components import Component, ComponentLoadContext +from dagster_components.core.component import component +from dagster_components.core.component_decl_builder import ComponentDeclNode, YamlComponentDecl +from dagster_components.core.dsl_schema import OpSpecBaseModel + + +class DbtNodeTranslatorParams(BaseModel): + key: Optional[str] = None + group: Optional[str] = None + + +class DbtProjectParams(BaseModel): + dbt: DbtCliResource + op: Optional[OpSpecBaseModel] = None + translator: Optional[DbtNodeTranslatorParams] = None + + +class DbtGenerateParams(BaseModel): + init: bool = Field(default=False) + project_path: Optional[str] = None + + @staticmethod + @click.command + @click.option("--project-path", "-p", type=click.Path(resolve_path=True), default=None) + @click.option("--init", "-i", is_flag=True, default=False) + def cli(project_path: Optional[str], init: bool) -> "DbtGenerateParams": + return DbtGenerateParams(project_path=project_path, init=init) + + +class DbtProjectComponentTranslator(DagsterDbtTranslator): + def __init__( + self, + *, + translator_params: Optional[DbtNodeTranslatorParams] = None, + ): + self.translator_params = translator_params + + def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> AssetKey: + if not self.translator_params or not self.translator_params.key: + return super().get_asset_key(dbt_resource_props) + + return AssetKey.from_user_string( + Template(self.translator_params.key).render(node=dbt_resource_props) + ) + + def get_group_name(self, dbt_resource_props) -> Optional[str]: + if not self.translator_params or not self.translator_params.group: + return super().get_group_name(dbt_resource_props) + + return Template(self.translator_params.group).render(node=dbt_resource_props) + + +@component(name="dbt_project") +class DbtProjectComponent(Component): + params_schema = DbtProjectParams + generate_params_schema = DbtGenerateParams + + def __init__( + self, + dbt_resource: DbtCliResource, + op_spec: Optional[OpSpecBaseModel], + dbt_translator: Optional[DagsterDbtTranslator], + ): + self.dbt_resource = dbt_resource + self.op_spec = op_spec + self.dbt_translator = dbt_translator + + @classmethod + def from_decl_node(cls, context: ComponentLoadContext, decl_node: ComponentDeclNode) -> Self: + assert isinstance(decl_node, YamlComponentDecl) + + # all paths should be resolved relative to the directory we're in + with pushd(str(decl_node.path)): + loaded_params = TypeAdapter(cls.params_schema).validate_python( + decl_node.component_file_model.params + ) + return cls( + dbt_resource=loaded_params.dbt, + op_spec=loaded_params.op, + dbt_translator=DbtProjectComponentTranslator( + translator_params=loaded_params.translator + ), + ) + + def build_defs(self, context: ComponentLoadContext) -> Definitions: + project = DbtProject(self.dbt_resource.project_dir) + project.prepare_if_dev() + + @dbt_assets( + manifest=project.manifest_path, + project=project, + name=self.op_spec.name if self.op_spec else project.name, + op_tags=self.op_spec.tags if self.op_spec else None, + dagster_dbt_translator=self.dbt_translator, + ) + def _fn(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + return Definitions(assets=[_fn], resources={"dbt": self.dbt_resource}) + + @classmethod + def generate_files(cls, params: DbtGenerateParams) -> Mapping[str, Any]: + cwd = os.getcwd() + if params.project_path: + # NOTE: CWD is not set "correctly" above so we prepend "../../.." as a temporary hack to + # make sure the path is right. + relative_path = os.path.join( + "../../../", os.path.relpath(params.project_path, start=cwd) + ) + elif params.init: + dbtRunner().invoke(["init"]) + subpaths = [ + path for path in Path(cwd).iterdir() if path.is_dir() and path.name != "logs" + ] + check.invariant(len(subpaths) == 1, "Expected exactly one subpath to be created.") + # this path should be relative to this directory + relative_path = subpaths[0].name + else: + relative_path = None + + return {"dbt": {"project_dir": relative_path}} diff --git a/python_modules/libraries/dagster-components/dagster_components/impls/pipes_subprocess_script_collection.py b/python_modules/libraries/dagster-components/dagster_components/lib/pipes_subprocess_script_collection.py similarity index 94% rename from python_modules/libraries/dagster-components/dagster_components/impls/pipes_subprocess_script_collection.py rename to python_modules/libraries/dagster-components/dagster_components/lib/pipes_subprocess_script_collection.py index e43310e534a4c..cc461b6b2b234 100644 --- a/python_modules/libraries/dagster-components/dagster_components/impls/pipes_subprocess_script_collection.py +++ b/python_modules/libraries/dagster-components/dagster_components/lib/pipes_subprocess_script_collection.py @@ -11,7 +11,12 @@ from dagster._utils.warnings import suppress_dagster_warnings from pydantic import BaseModel, TypeAdapter -from dagster_components.core.component import Component, ComponentDeclNode, ComponentLoadContext +from dagster_components.core.component import ( + Component, + ComponentDeclNode, + ComponentLoadContext, + component, +) from dagster_components.core.component_decl_builder import YamlComponentDecl if TYPE_CHECKING: @@ -48,6 +53,7 @@ class PipesSubprocessScriptCollectionParams(BaseModel): scripts: Sequence[PipesSubprocessScriptParams] +@component(name="pipes_subprocess_script_collection") class PipesSubprocessScriptCollection(Component): params_schema = PipesSubprocessScriptCollectionParams @@ -68,7 +74,7 @@ def from_decl_node( ) -> "PipesSubprocessScriptCollection": assert isinstance(component_decl, YamlComponentDecl) loaded_params = TypeAdapter(cls.params_schema).validate_python( - component_decl.defs_file_model.component_params + component_decl.component_file_model.params ) path_specs = {} diff --git a/python_modules/libraries/dagster-components/dagster_components/impls/sling_replication.py b/python_modules/libraries/dagster-components/dagster_components/lib/sling_replication.py similarity index 63% rename from python_modules/libraries/dagster-components/dagster_components/impls/sling_replication.py rename to python_modules/libraries/dagster-components/dagster_components/lib/sling_replication.py index ec57b57055fe9..772cc42d0e1ec 100644 --- a/python_modules/libraries/dagster-components/dagster_components/impls/sling_replication.py +++ b/python_modules/libraries/dagster-components/dagster_components/lib/sling_replication.py @@ -1,49 +1,47 @@ import os from pathlib import Path -from typing import Dict, Optional +from typing import Any, Iterator, Optional, Union import yaml from dagster._core.definitions.definitions_class import Definitions +from dagster._core.definitions.events import AssetMaterialization +from dagster._core.definitions.result import MaterializeResult from dagster_embedded_elt.sling import SlingResource, sling_assets from dagster_embedded_elt.sling.resources import AssetExecutionContext from pydantic import BaseModel, TypeAdapter from typing_extensions import Self from dagster_components import Component, ComponentLoadContext +from dagster_components.core.component import component from dagster_components.core.component_decl_builder import ComponentDeclNode, YamlComponentDecl - - -class OpSpecBaseModel(BaseModel): - name: Optional[str] = None - tags: Optional[Dict[str, str]] = None +from dagster_components.core.dsl_schema import OpSpecBaseModel class SlingReplicationParams(BaseModel): - sling: SlingResource + sling: Optional[SlingResource] = None op: Optional[OpSpecBaseModel] = None +@component(name="sling_replication") class SlingReplicationComponent(Component): params_schema = SlingReplicationParams - def __init__( - self, dirpath: Path, resource: SlingResource, op_spec: Optional[OpSpecBaseModel] = None - ): + def __init__(self, dirpath: Path, resource: SlingResource, op_spec: Optional[OpSpecBaseModel]): self.dirpath = dirpath self.resource = resource self.op_spec = op_spec - @classmethod - def registered_name(cls) -> str: - return "sling_replication" - @classmethod def from_decl_node(cls, context: ComponentLoadContext, decl_node: ComponentDeclNode) -> Self: assert isinstance(decl_node, YamlComponentDecl) loaded_params = TypeAdapter(cls.params_schema).validate_python( - decl_node.defs_file_model.component_params + decl_node.component_file_model.params + ) + return cls( + dirpath=decl_node.path, + resource=loaded_params.sling or SlingResource(), + op_spec=loaded_params.op, ) - return cls(dirpath=decl_node.path, resource=loaded_params.sling, op_spec=loaded_params.op) def build_defs(self, context: ComponentLoadContext) -> Definitions: @sling_assets( @@ -52,15 +50,20 @@ def build_defs(self, context: ComponentLoadContext) -> Definitions: replication_config=self.dirpath / "replication.yaml", ) def _fn(context: AssetExecutionContext, sling: SlingResource): - yield from sling.replicate(context=context) + yield from self.execute(context=context, sling=sling) return Definitions(assets=[_fn], resources={"sling": self.resource}) @classmethod - def generate_files(cls) -> None: + def generate_files(cls, params: Any) -> None: replication_path = Path(os.getcwd()) / "replication.yaml" with open(replication_path, "w") as f: yaml.dump( {"source": {}, "target": {}, "streams": {}}, f, ) + + def execute( + self, context: AssetExecutionContext, sling: SlingResource + ) -> Iterator[Union[AssetMaterialization, MaterializeResult]]: + yield from sling.replicate(context=context) diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/__init__.py b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/__init__.py new file mode 100644 index 0000000000000..8b137891791fe --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/__init__.py @@ -0,0 +1 @@ + diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/component.py b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/component.py new file mode 100644 index 0000000000000..0019f56f18974 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/component.py @@ -0,0 +1,12 @@ +from typing import Iterator + +from dagster._core.execution.context.asset_execution_context import AssetExecutionContext +from dagster_components import component +from dagster_components.lib.sling_replication import SlingReplicationComponent +from dagster_embedded_elt.sling import SlingResource + + +@component(name="debug_sling_replication") +class DebugSlingReplicationComponent(SlingReplicationComponent): + def execute(self, context: AssetExecutionContext, sling: SlingResource) -> Iterator: + return sling.replicate(context=context, debug=True) diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/component.yaml b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/component.yaml new file mode 100644 index 0000000000000..c9ba8c6cb4c4c --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/component.yaml @@ -0,0 +1,7 @@ +type: .debug_sling_replication + +params: + sling: + connections: + - name: snowflake + type: snowflake diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/replication.yaml b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/replication.yaml new file mode 100644 index 0000000000000..611c14c59bd3b --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/components/debug_sling_component/replication.yaml @@ -0,0 +1,24 @@ +source: MY_SOURCE +target: MY_TARGET + +defaults: + mode: full-refresh + object: "{stream_schema}_{stream_table}" + +streams: + public.accounts: + public.users: + public.finance_departments_old: + object: "departments" # overwrite default object + source_options: + empty_as_null: false + public."Transactions": + mode: incremental # overwrite default mode + primary_key: id + update_key: last_updated_at + + public.all_users: + sql: | + select all_user_id, name + from public."all_Users" + object: public.all_users # need to add 'object' key for custom SQL diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/definitions.py b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/definitions.py new file mode 100644 index 0000000000000..dd49b214d9757 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/definitions.py @@ -0,0 +1,11 @@ +from pathlib import Path + +from dagster._core.definitions.definitions_class import Definitions +from dagster_components.core.component_defs_builder import ( + build_defs_from_toplevel_components_folder, +) + +defs = build_defs_from_toplevel_components_folder(path=Path(__file__).parent) + +if __name__ == "__main__": + Definitions.validate_loadable(defs) diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/lib/__init__.py b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/lib/__init__.py new file mode 100644 index 0000000000000..8b137891791fe --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location/lib/__init__.py @@ -0,0 +1 @@ + diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location_tests/__init__.py b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location_tests/__init__.py new file mode 100644 index 0000000000000..8b137891791fe --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/custom_sling_location_tests/__init__.py @@ -0,0 +1 @@ + diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/pyproject.toml.jinja b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/pyproject.toml similarity index 61% rename from python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/pyproject.toml.jinja rename to python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/pyproject.toml index 0fa5131cb4abc..c135dc0fccf0c 100644 --- a/python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/pyproject.toml.jinja +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/pyproject.toml @@ -1,5 +1,5 @@ [project] -name = "{{ project_name }}" +name = "custom_sling_location" requires-python = ">=3.9,<3.13" version = "0.1.0" dependencies = [] @@ -12,8 +12,8 @@ requires = ["setuptools"] build-backend = "setuptools.build_meta" [tool.dagster] -module_name = "{{ project_name }}.definitions" -project_name = "{{ project_name }}" +module_name = "custom_sling_location.definitions" +project_name = "custom_sling_location" [tool.setuptools.packages.find] -exclude=["{{ project_name }}_tests"] +exclude=["custom_sling_location_tests"] diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/uv.lock b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/uv.lock new file mode 100644 index 0000000000000..d307093b91d06 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/custom_sling_location/uv.lock @@ -0,0 +1,7 @@ +version = 1 +requires-python = ">=3.9, <3.13" + +[[package]] +name = "custom-sling-location" +version = "0.1.0" +source = { editable = "." } diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/python_script_location/components/scripts/defs.yml b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/python_script_location/components/scripts/component.yaml similarity index 78% rename from python_modules/libraries/dagster-components/dagster_components_tests/code_locations/python_script_location/components/scripts/defs.yml rename to python_modules/libraries/dagster-components/dagster_components_tests/code_locations/python_script_location/components/scripts/component.yaml index 3fda5a1194652..34bd590951b8b 100644 --- a/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/python_script_location/components/scripts/defs.yml +++ b/python_modules/libraries/dagster-components/dagster_components_tests/code_locations/python_script_location/components/scripts/component.yaml @@ -1,6 +1,6 @@ -component_type: pipes_subprocess_script_collection +type: dagster_components.pipes_subprocess_script_collection -component_params: +params: scripts: - path: script_one.py assets: diff --git a/examples/experimental/dagster-components/dagster_components_tests/integration_tests/test_dbt_project.py b/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_dbt_project.py similarity index 77% rename from examples/experimental/dagster-components/dagster_components_tests/integration_tests/test_dbt_project.py rename to python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_dbt_project.py index bd64c916dd156..d3bc69c881f32 100644 --- a/examples/experimental/dagster-components/dagster_components_tests/integration_tests/test_dbt_project.py +++ b/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_dbt_project.py @@ -6,13 +6,13 @@ import pytest from dagster import AssetKey -from dagster_components.core.component_decl_builder import DefsFileModel +from dagster_components.core.component_decl_builder import ComponentFileModel from dagster_components.core.component_defs_builder import ( YamlComponentDecl, build_components_from_component_folder, defs_from_components, ) -from dagster_components.impls.dbt_project import DbtProjectComponent +from dagster_components.lib.dbt_project import DbtProjectComponent from dagster_dbt import DbtProject from dagster_components_tests.utils import assert_assets, get_asset_keys, script_load_context @@ -48,13 +48,19 @@ def test_python_params(dbt_path: Path) -> None: context=script_load_context(), decl_node=YamlComponentDecl( path=dbt_path / COMPONENT_RELPATH, - defs_file_model=DefsFileModel( - component_type="dbt_project", - component_params={"dbt": {"project_dir": "jaffle_shop"}}, + component_file_model=ComponentFileModel( + type="dbt_project", + params={ + "dbt": {"project_dir": "jaffle_shop"}, + "op": {"name": "some_op", "tags": {"tag1": "value"}}, + }, ), ), ) assert get_asset_keys(component) == JAFFLE_SHOP_KEYS + defs = component.build_defs(script_load_context()) + assert defs.get_assets_def("stg_customers").op.name == "some_op" + assert defs.get_assets_def("stg_customers").op.tags["tag1"] == "value" def test_load_from_path(dbt_path: Path) -> None: diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_sling_integration_test.py b/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_sling_integration_test.py index 0ecec6b7d3cbb..494e4255aa853 100644 --- a/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_sling_integration_test.py +++ b/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_sling_integration_test.py @@ -2,18 +2,22 @@ import tempfile from contextlib import contextmanager from pathlib import Path -from typing import Any, Dict, Generator, Mapping +from typing import Any, Dict, Generator, Iterator, Mapping, Union import pytest import yaml from dagster import AssetKey +from dagster._core.definitions.events import AssetMaterialization +from dagster._core.definitions.result import MaterializeResult +from dagster._core.execution.context.asset_execution_context import AssetExecutionContext from dagster._utils.env import environ -from dagster_components.core.component_decl_builder import DefsFileModel +from dagster_components.core.component_decl_builder import ComponentFileModel from dagster_components.core.component_defs_builder import ( YamlComponentDecl, build_components_from_component_folder, ) -from dagster_components.impls.sling_replication import SlingReplicationComponent +from dagster_components.lib.sling_replication import SlingReplicationComponent, component +from dagster_embedded_elt.sling import SlingResource from dagster_components_tests.utils import assert_assets, get_asset_keys, script_load_context @@ -32,7 +36,7 @@ def _update_yaml(path: Path, fn) -> None: @contextmanager @pytest.fixture(scope="module") def sling_path() -> Generator[Path, None, None]: - """Sets up a temporary directory with a replication.yaml and defs.yml file that reference + """Sets up a temporary directory with a replication.yaml and component.yaml file that reference the proper temp path. """ with tempfile.TemporaryDirectory() as temp_dir: @@ -50,12 +54,10 @@ def _update_replication(data: Dict[str, Any]) -> Mapping[str, Any]: _update_yaml(replication_path, _update_replication) # update the defs yaml to add a duckdb instance - defs_path = Path(temp_dir) / COMPONENT_RELPATH / "defs.yml" + defs_path = Path(temp_dir) / COMPONENT_RELPATH / "component.yaml" def _update_defs(data: Dict[str, Any]) -> Mapping[str, Any]: - data["component_params"]["sling"]["connections"][0]["instance"] = ( - f"{temp_dir}/duckdb" - ) + data["params"]["sling"]["connections"][0]["instance"] = f"{temp_dir}/duckdb" return data _update_yaml(defs_path, _update_defs) @@ -69,9 +71,9 @@ def test_python_params(sling_path: Path) -> None: context=context, decl_node=YamlComponentDecl( path=sling_path / COMPONENT_RELPATH, - defs_file_model=DefsFileModel( - component_type="sling_replication", - component_params={"sling": {}}, + component_file_model=ComponentFileModel( + type="sling_replication", + params={"sling": {}}, ), ), ) @@ -92,9 +94,9 @@ def test_python_params_op_name(sling_path: Path) -> None: context=context, decl_node=YamlComponentDecl( path=sling_path / COMPONENT_RELPATH, - defs_file_model=DefsFileModel( - component_type="sling_replication", - component_params={"sling": {}, "op": {"name": "my_op"}}, + component_file_model=ComponentFileModel( + type="sling_replication", + params={"sling": {}, "op": {"name": "my_op"}}, ), ), ) @@ -115,9 +117,9 @@ def test_python_params_op_tags(sling_path: Path) -> None: context=context, decl_node=YamlComponentDecl( path=sling_path / COMPONENT_RELPATH, - defs_file_model=DefsFileModel( - component_type="sling_replication", - component_params={"sling": {}, "op": {"tags": {"tag1": "value1"}}}, + component_file_model=ComponentFileModel( + type="sling_replication", + params={"sling": {}, "op": {"tags": {"tag1": "value1"}}}, ), ), ) @@ -138,3 +140,27 @@ def test_load_from_path(sling_path: Path) -> None: } assert_assets(components[0], 2) + + +def test_sling_subclass() -> None: + @component(name="debug_sling_replication") + class DebugSlingReplicationComponent(SlingReplicationComponent): + def execute( + self, context: AssetExecutionContext, sling: SlingResource + ) -> Iterator[Union[AssetMaterialization, MaterializeResult]]: + return sling.replicate(context=context, debug=True) + + component_inst = DebugSlingReplicationComponent.from_decl_node( + context=script_load_context(), + decl_node=YamlComponentDecl( + path=STUB_LOCATION_PATH / COMPONENT_RELPATH, + component_file_model=ComponentFileModel( + type="debug_sling_replication", + params={"sling": {}}, + ), + ), + ) + assert get_asset_keys(component_inst) == { + AssetKey("input_csv"), + AssetKey("input_duckdb"), + } diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_templated_custom_keys_dbt_project.py b/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_templated_custom_keys_dbt_project.py new file mode 100644 index 0000000000000..5b21bed30e642 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/integration_tests/test_templated_custom_keys_dbt_project.py @@ -0,0 +1,120 @@ +import shutil +import tempfile +from contextlib import contextmanager +from pathlib import Path +from typing import TYPE_CHECKING, Generator + +import pytest +from dagster import AssetKey +from dagster_components.core.component_decl_builder import ComponentFileModel +from dagster_components.core.component_defs_builder import ( + YamlComponentDecl, + build_components_from_component_folder, + defs_from_components, +) +from dagster_components.lib.dbt_project import DbtProjectComponent +from dagster_dbt import DbtProject + +from dagster_components_tests.utils import assert_assets, get_asset_keys, script_load_context + +if TYPE_CHECKING: + from dagster._core.definitions.definitions_class import Definitions + +STUB_LOCATION_PATH = ( + Path(__file__).parent.parent + / "stub_code_locations" + / "templated_custom_keys_dbt_project_location" +) +COMPONENT_RELPATH = "components/jaffle_shop_dbt" + +JAFFLE_SHOP_KEYS = { + AssetKey("customers"), + AssetKey("orders"), + AssetKey("raw_customers"), + AssetKey("raw_orders"), + AssetKey("raw_payments"), + AssetKey("stg_customers"), + AssetKey("stg_orders"), + AssetKey("stg_payments"), +} + +JAFFLE_SHOP_KEYS_WITH_PREFIX = { + AssetKey(["some_prefix", "customers"]), + AssetKey(["some_prefix", "orders"]), + AssetKey(["some_prefix", "raw_customers"]), + AssetKey(["some_prefix", "raw_orders"]), + AssetKey(["some_prefix", "raw_payments"]), + AssetKey(["some_prefix", "stg_customers"]), + AssetKey(["some_prefix", "stg_orders"]), + AssetKey(["some_prefix", "stg_payments"]), +} + + +@contextmanager +@pytest.fixture(scope="module") +def dbt_path() -> Generator[Path, None, None]: + with tempfile.TemporaryDirectory() as temp_dir: + shutil.copytree(STUB_LOCATION_PATH, temp_dir, dirs_exist_ok=True) + # make sure a manifest.json file is created + project = DbtProject(Path(temp_dir) / "components/jaffle_shop_dbt/jaffle_shop") + project.preparer.prepare(project) + yield Path(temp_dir) + + +def test_python_params_node_rename(dbt_path: Path) -> None: + component = DbtProjectComponent.from_decl_node( + context=script_load_context(), + decl_node=YamlComponentDecl( + path=dbt_path / COMPONENT_RELPATH, + component_file_model=ComponentFileModel( + type="dbt_project", + params={ + "dbt": {"project_dir": "jaffle_shop"}, + "translator": { + "key": "some_prefix/{{ node.name }}", + }, + }, + ), + ), + ) + assert get_asset_keys(component) == JAFFLE_SHOP_KEYS_WITH_PREFIX + + +def test_python_params_group(dbt_path: Path) -> None: + comp = DbtProjectComponent.from_decl_node( + context=script_load_context(), + decl_node=YamlComponentDecl( + path=dbt_path / COMPONENT_RELPATH, + component_file_model=ComponentFileModel( + type="dbt_project", + params={ + "dbt": {"project_dir": "jaffle_shop"}, + "translator": { + "group": "some_group", + }, + }, + ), + ), + ) + assert get_asset_keys(comp) == JAFFLE_SHOP_KEYS + defs: Definitions = comp.build_defs(script_load_context()) + for key in get_asset_keys(comp): + assert defs.get_assets_def(key).get_asset_spec(key).group_name == "some_group" + + +def test_load_from_path(dbt_path: Path) -> None: + components = build_components_from_component_folder( + script_load_context(), dbt_path / "components" + ) + assert len(components) == 1 + assert get_asset_keys(components[0]) == JAFFLE_SHOP_KEYS_WITH_PREFIX + + assert_assets(components[0], len(JAFFLE_SHOP_KEYS_WITH_PREFIX)) + + defs = defs_from_components( + context=script_load_context(), + components=components, + resources={}, + ) + + assert defs.get_asset_graph().get_all_asset_keys() == JAFFLE_SHOP_KEYS_WITH_PREFIX diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/dbt_project_location/components/jaffle_shop_dbt/component.yaml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/dbt_project_location/components/jaffle_shop_dbt/component.yaml new file mode 100644 index 0000000000000..85c04ada2fb8d --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/dbt_project_location/components/jaffle_shop_dbt/component.yaml @@ -0,0 +1,5 @@ +type: dagster_components.dbt_project + +params: + dbt: + project_dir: jaffle_shop diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/dbt_project_location/components/jaffle_shop_dbt/defs.yml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/dbt_project_location/components/jaffle_shop_dbt/defs.yml deleted file mode 100644 index a8eeb5893f3a7..0000000000000 --- a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/dbt_project_location/components/jaffle_shop_dbt/defs.yml +++ /dev/null @@ -1,5 +0,0 @@ -component_type: dbt_project - -component_params: - dbt: - project_dir: jaffle_shop diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/sling_location/components/ingest/defs.yml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/sling_location/components/ingest/component.yaml similarity index 66% rename from python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/sling_location/components/ingest/defs.yml rename to python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/sling_location/components/ingest/component.yaml index 9b7b358929e74..e92d6ec0ad839 100644 --- a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/sling_location/components/ingest/defs.yml +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/sling_location/components/ingest/component.yaml @@ -1,6 +1,6 @@ -component_type: sling_replication +type: dagster_components.sling_replication -component_params: +params: sling: connections: - name: DUCKDB diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/component.yaml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/component.yaml new file mode 100644 index 0000000000000..07a7bfff6f501 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/component.yaml @@ -0,0 +1,8 @@ +type: dagster_components.dbt_project + +params: + dbt: + project_dir: jaffle_shop + + translator: + key: "some_prefix/{{node.name}}" diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/.gitignore b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/.gitignore new file mode 100644 index 0000000000000..c12c7f9624ad6 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/.gitignore @@ -0,0 +1,13 @@ + +target/ +dbt_packages/ +dbt_modules/ +logs/ +**/.DS_Store +.user.yml +venv/ +env/ +**/*.duckdb +**/*.duckdb.wal +tmp/ +state/ diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/dbt_project.yml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/dbt_project.yml new file mode 100644 index 0000000000000..377ac5fae4c90 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/dbt_project.yml @@ -0,0 +1,26 @@ +name: "jaffle_shop" + +config-version: 2 +version: "0.1" + +profile: "jaffle_shop" + +model-paths: ["models"] +seed-paths: ["seeds"] +test-paths: ["tests"] +analysis-paths: ["analysis"] +macro-paths: ["macros"] + +target-path: "target" +clean-targets: + - "target" + - "dbt_modules" + - "logs" + +require-dbt-version: [">=1.0.0", "<2.0.0"] + +models: + jaffle_shop: + materialized: table + staging: + materialized: view diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/customers.sql b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/customers.sql new file mode 100644 index 0000000000000..00fed3033006a --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/customers.sql @@ -0,0 +1,75 @@ +with customers as ( + + select * from {{ ref('stg_customers') }} + +), + +orders as ( + + select * from {{ ref('stg_orders') }} + +), + +payments as ( + + select * from {{ ref('stg_payments') }} + +), + +customer_orders as ( + + select + customer_id, + + min(order_date) as first_order, + max(order_date) as most_recent_order, + count(order_id) as number_of_orders + from orders + + group by customer_id + +), + +customer_payments as ( + + select + orders.customer_id, + sum(amount) as total_amount + + from payments + + left join orders on + payments.order_id = orders.order_id + + group by orders.customer_id + +), + +final as ( + + select + customers.customer_id, + customers.first_name, + customers.last_name, + customer_orders.first_order, + customer_orders.most_recent_order, + customer_orders.number_of_orders, + customer_payments.total_amount as customer_lifetime_value + + from customers + + left join customer_orders + on customers.customer_id = customer_orders.customer_id + + left join customer_payments + on customers.customer_id = customer_payments.customer_id + +) + +select * from final + +{% if var('break_customer_build', 'false') == 'true' %} + +does not work + +{% endif %} diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/docs.md b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/docs.md new file mode 100644 index 0000000000000..d990aad5a92ac --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/docs.md @@ -0,0 +1,13 @@ +{% docs orders_status %} + +Orders can be one of the following statuses: + +| status | description | +| -------------- | ---------------------------------------------------------------------------------------------------------------------- | +| placed | The order has been placed but has not yet left the warehouse | +| shipped | The order has ben shipped to the customer and is currently in transit | +| completed | The order has been received by the customer | +| return_pending | The customer has indicated that they would like to return the order, but it has not yet been received at the warehouse | +| returned | The order has been returned by the customer and received at the warehouse | + +{% enddocs %} diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/orders.sql b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/orders.sql new file mode 100644 index 0000000000000..cbb2934911821 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/orders.sql @@ -0,0 +1,56 @@ +{% set payment_methods = ['credit_card', 'coupon', 'bank_transfer', 'gift_card'] %} + +with orders as ( + + select * from {{ ref('stg_orders') }} + +), + +payments as ( + + select * from {{ ref('stg_payments') }} + +), + +order_payments as ( + + select + order_id, + + {% for payment_method in payment_methods -%} + sum(case when payment_method = '{{ payment_method }}' then amount else 0 end) as {{ payment_method }}_amount, + {% endfor -%} + + sum(amount) as total_amount + + from payments + + group by order_id + +), + +final as ( + + select + orders.order_id, + orders.customer_id, + orders.order_date, + orders.status, + + {% for payment_method in payment_methods -%} + + order_payments.{{ payment_method }}_amount, + + {% endfor -%} + + order_payments.total_amount as amount + + from orders + + + left join order_payments + on orders.order_id = order_payments.order_id + +) + +select * from final diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/overview.md b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/overview.md new file mode 100644 index 0000000000000..0544c42b17d5e --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/overview.md @@ -0,0 +1,11 @@ +{% docs __overview__ %} + +## Data Documentation for Jaffle Shop + +`jaffle_shop` is a fictional ecommerce store. + +This [dbt](https://www.getdbt.com/) project is for testing out code. + +The source code can be found [here](https://github.com/clrcrl/jaffle_shop). + +{% enddocs %} diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/schema.yml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/schema.yml new file mode 100644 index 0000000000000..05c5be6930f7f --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/schema.yml @@ -0,0 +1,83 @@ +version: 2 + +models: + - name: customers + description: This table has basic information about a customer, as well as some derived facts based on a customer's orders + + columns: + - name: customer_id + description: This is a unique identifier for a customer + tests: + - unique + - not_null + + - name: first_name + description: Customer's first name. PII. + + - name: last_name + description: Customer's last name. PII. + + - name: first_order + description: Date (UTC) of a customer's first order + + - name: most_recent_order + description: Date (UTC) of a customer's most recent order + + - name: number_of_orders + description: Count of the number of orders a customer has placed + + - name: total_order_amount + description: Total value (AUD) of a customer's orders + + - name: orders + description: This table has basic information about orders, as well as some derived facts based on payments + + columns: + - name: order_id + tests: + - unique + - not_null + description: This is a unique identifier for an order + + - name: customer_id + description: Foreign key to the customers table + tests: + - not_null + - relationships: + to: ref('customers') + field: customer_id + + - name: order_date + description: Date (UTC) that the order was placed + + - name: status + description: '{{ doc("orders_status") }}' + tests: + - accepted_values: + values: + ["placed", "shipped", "completed", "return_pending", "returned"] + + - name: amount + description: Total amount (AUD) of the order + tests: + - not_null + + - name: credit_card_amount + description: Amount of the order (AUD) paid for by credit card + tests: + - not_null + + - name: coupon_amount + description: Amount of the order (AUD) paid for by coupon + tests: + - not_null + + - name: bank_transfer_amount + description: Amount of the order (AUD) paid for by bank transfer + tests: + - not_null + + - name: gift_card_amount + description: Amount of the order (AUD) paid for by gift card + tests: + - not_null diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/schema.yml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/schema.yml new file mode 100644 index 0000000000000..258345211a9a2 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/schema.yml @@ -0,0 +1,32 @@ +version: 2 + +models: + - name: stg_customers + columns: + - name: customer_id + tests: + - unique + - not_null + + - name: stg_orders + columns: + - name: order_id + tests: + - unique + - not_null + - name: status + tests: + - accepted_values: + values: + ["placed", "shipped", "completed", "return_pending", "returned"] + + - name: stg_payments + columns: + - name: payment_id + tests: + - unique + - not_null + - name: payment_method + tests: + - accepted_values: + values: ["credit_card", "coupon", "bank_transfer", "gift_card"] diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_customers.sql b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_customers.sql new file mode 100644 index 0000000000000..cad047269506e --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_customers.sql @@ -0,0 +1,22 @@ +with source as ( + + {#- + Normally we would select from the table here, but we are using seeds to load + our data in this project + #} + select * from {{ ref('raw_customers') }} + +), + +renamed as ( + + select + id as customer_id, + first_name, + last_name + + from source + +) + +select * from renamed diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_orders.sql b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_orders.sql new file mode 100644 index 0000000000000..a654dcb9474a7 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_orders.sql @@ -0,0 +1,23 @@ +with source as ( + + {#- + Normally we would select from the table here, but we are using seeds to load + our data in this project + #} + select * from {{ ref('raw_orders') }} + +), + +renamed as ( + + select + id as order_id, + user_id as customer_id, + order_date, + status + + from source + +) + +select * from renamed diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_payments.sql b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_payments.sql new file mode 100644 index 0000000000000..700cf7f4f657b --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/models/staging/stg_payments.sql @@ -0,0 +1,25 @@ +with source as ( + + {#- + Normally we would select from the table here, but we are using seeds to load + our data in this project + #} + select * from {{ ref('raw_payments') }} + +), + +renamed as ( + + select + id as payment_id, + order_id, + payment_method, + + -- `amount` is currently stored in cents, so we convert it to dollars + amount / 100 as amount + + from source + +) + +select * from renamed diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/profiles.yml b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/profiles.yml new file mode 100644 index 0000000000000..b7349909fcbb1 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/profiles.yml @@ -0,0 +1,23 @@ +jaffle_shop: + target: dev + outputs: + dev: + type: duckdb + path: "{{ env_var('DAGSTER_DBT_PYTEST_XDIST_DUCKDB_DBFILE_PATH', 'target/local.duckdb') }}" + schema: "{{ env_var('DAGSTER_DBT_JAFFLE_SCHEMA', 'dev') }}" + threads: 24 + snowflake: + type: snowflake + account: "{{ env_var('SNOWFLAKE_ACCOUNT') }}" + user: "{{ env_var('SNOWFLAKE_USER') }}" + password: "{{ env_var('SNOWFLAKE_PASSWORD') }}" + client_session_keep_alive: False + database: TESTDB + schema: TESTSCHEMA + bigquery: + type: bigquery + method: service-account + project: "{{ env_var('GCP_PROJECT_ID') }}" + dataset: BIGQUERY_IO_MANAGER_SCHEMA + threads: 4 + keyfile: "{{ env_var('GOOGLE_APPLICATION_CREDENTIALS') }}" diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/components/.gitkeep b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/.gitkeep similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/components/.gitkeep rename to python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/.gitkeep diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_customers.csv b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_customers.csv new file mode 100644 index 0000000000000..b3e6747d69a20 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_customers.csv @@ -0,0 +1,101 @@ +id,first_name,last_name +1,Michael,P. +2,Shawn,M. +3,Kathleen,P. +4,Jimmy,C. +5,Katherine,R. +6,Sarah,R. +7,Martin,M. +8,Frank,R. +9,Jennifer,F. +10,Henry,W. +11,Fred,S. +12,Amy,D. +13,Kathleen,M. +14,Steve,F. +15,Teresa,H. +16,Amanda,H. +17,Kimberly,R. +18,Johnny,K. +19,Virginia,F. +20,Anna,A. +21,Willie,H. +22,Sean,H. +23,Mildred,A. +24,David,G. +25,Victor,H. +26,Aaron,R. +27,Benjamin,B. +28,Lisa,W. +29,Benjamin,K. +30,Christina,W. +31,Jane,G. +32,Thomas,O. +33,Katherine,M. +34,Jennifer,S. +35,Sara,T. +36,Harold,O. +37,Shirley,J. +38,Dennis,J. +39,Louise,W. +40,Maria,A. +41,Gloria,C. +42,Diana,S. +43,Kelly,N. +44,Jane,R. +45,Scott,B. +46,Norma,C. +47,Marie,P. +48,Lillian,C. +49,Judy,N. +50,Billy,L. +51,Howard,R. +52,Laura,F. +53,Anne,B. +54,Rose,M. +55,Nicholas,R. +56,Joshua,K. +57,Paul,W. +58,Kathryn,K. +59,Adam,A. +60,Norma,W. +61,Timothy,R. +62,Elizabeth,P. +63,Edward,G. +64,David,C. +65,Brenda,W. +66,Adam,W. +67,Michael,H. +68,Jesse,E. +69,Janet,P. +70,Helen,F. +71,Gerald,C. +72,Kathryn,O. +73,Alan,B. +74,Harry,A. +75,Andrea,H. +76,Barbara,W. +77,Anne,W. +78,Harry,H. +79,Jack,R. +80,Phillip,H. +81,Shirley,H. +82,Arthur,D. +83,Virginia,R. +84,Christina,R. +85,Theresa,M. +86,Jason,C. +87,Phillip,B. +88,Adam,T. +89,Margaret,J. +90,Paul,P. +91,Todd,W. +92,Willie,O. +93,Frances,R. +94,Gregory,H. +95,Lisa,P. +96,Jacqueline,A. +97,Shirley,D. +98,Nicole,M. +99,Mary,G. +100,Jean,M. diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_orders.csv b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_orders.csv new file mode 100644 index 0000000000000..7c2be078882c5 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_orders.csv @@ -0,0 +1,100 @@ +id,user_id,order_date,status +1,1,2018-01-01,returned +2,3,2018-01-02,completed +3,94,2018-01-04,completed +4,50,2018-01-05,completed +5,64,2018-01-05,completed +6,54,2018-01-07,completed +7,88,2018-01-09,completed +8,2,2018-01-11,returned +9,53,2018-01-12,completed +10,7,2018-01-14,completed +11,99,2018-01-14,completed +12,59,2018-01-15,completed +13,84,2018-01-17,completed +14,40,2018-01-17,returned +15,25,2018-01-17,completed +16,39,2018-01-18,completed +17,71,2018-01-18,completed +18,64,2018-01-20,returned +19,54,2018-01-22,completed +20,20,2018-01-23,completed +21,71,2018-01-23,completed +22,86,2018-01-24,completed +23,22,2018-01-26,return_pending +24,3,2018-01-27,completed +25,51,2018-01-28,completed +26,32,2018-01-28,completed +27,94,2018-01-29,completed +28,8,2018-01-29,completed +29,57,2018-01-31,completed +30,69,2018-02-02,completed +31,16,2018-02-02,completed +32,28,2018-02-04,completed +33,42,2018-02-04,completed +34,38,2018-02-06,completed +35,80,2018-02-08,completed +36,85,2018-02-10,completed +37,1,2018-02-10,completed +38,51,2018-02-10,completed +39,26,2018-02-11,completed +40,33,2018-02-13,completed +41,99,2018-02-14,completed +42,92,2018-02-16,completed +43,31,2018-02-17,completed +44,66,2018-02-17,completed +45,22,2018-02-17,completed +46,6,2018-02-19,completed +47,50,2018-02-20,completed +48,27,2018-02-21,completed +49,35,2018-02-21,completed +50,51,2018-02-23,completed +51,71,2018-02-24,completed +52,54,2018-02-25,return_pending +53,34,2018-02-26,completed +54,54,2018-02-26,completed +55,18,2018-02-27,completed +56,79,2018-02-28,completed +57,93,2018-03-01,completed +58,22,2018-03-01,completed +59,30,2018-03-02,completed +60,12,2018-03-03,completed +61,63,2018-03-03,completed +62,57,2018-03-05,completed +63,70,2018-03-06,completed +64,13,2018-03-07,completed +65,26,2018-03-08,completed +66,36,2018-03-10,completed +67,79,2018-03-11,completed +68,53,2018-03-11,completed +69,3,2018-03-11,completed +70,8,2018-03-12,completed +71,42,2018-03-12,shipped +72,30,2018-03-14,shipped +73,19,2018-03-16,completed +74,9,2018-03-17,shipped +75,69,2018-03-18,completed +76,25,2018-03-20,completed +77,35,2018-03-21,shipped +78,90,2018-03-23,shipped +79,52,2018-03-23,shipped +80,11,2018-03-23,shipped +81,76,2018-03-23,shipped +82,46,2018-03-24,shipped +83,54,2018-03-24,shipped +84,70,2018-03-26,placed +85,47,2018-03-26,shipped +86,68,2018-03-26,placed +87,46,2018-03-27,placed +88,91,2018-03-27,shipped +89,21,2018-03-28,placed +90,66,2018-03-30,shipped +91,47,2018-03-31,placed +92,84,2018-04-02,placed +93,66,2018-04-03,placed +94,63,2018-04-03,placed +95,27,2018-04-04,placed +96,90,2018-04-06,placed +97,89,2018-04-07,placed +98,41,2018-04-07,placed +99,85,2018-04-09,placed diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_payments.csv b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_payments.csv new file mode 100644 index 0000000000000..a587baab5955d --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/stub_code_locations/templated_custom_keys_dbt_project_location/components/jaffle_shop_dbt/jaffle_shop/seeds/raw_payments.csv @@ -0,0 +1,114 @@ +id,order_id,payment_method,amount +1,1,credit_card,1000 +2,2,credit_card,2000 +3,3,coupon,100 +4,4,coupon,2500 +5,5,bank_transfer,1700 +6,6,credit_card,600 +7,7,credit_card,1600 +8,8,credit_card,2300 +9,9,gift_card,2300 +10,9,bank_transfer,0 +11,10,bank_transfer,2600 +12,11,credit_card,2700 +13,12,credit_card,100 +14,13,credit_card,500 +15,13,bank_transfer,1400 +16,14,bank_transfer,300 +17,15,coupon,2200 +18,16,credit_card,1000 +19,17,bank_transfer,200 +20,18,credit_card,500 +21,18,credit_card,800 +22,19,gift_card,600 +23,20,bank_transfer,1500 +24,21,credit_card,1200 +25,22,bank_transfer,800 +26,23,gift_card,2300 +27,24,coupon,2600 +28,25,bank_transfer,2000 +29,25,credit_card,2200 +30,25,coupon,1600 +31,26,credit_card,3000 +32,27,credit_card,2300 +33,28,bank_transfer,1900 +34,29,bank_transfer,1200 +35,30,credit_card,1300 +36,31,credit_card,1200 +37,32,credit_card,300 +38,33,credit_card,2200 +39,34,bank_transfer,1500 +40,35,credit_card,2900 +41,36,bank_transfer,900 +42,37,credit_card,2300 +43,38,credit_card,1500 +44,39,bank_transfer,800 +45,40,credit_card,1400 +46,41,credit_card,1700 +47,42,coupon,1700 +48,43,gift_card,1800 +49,44,gift_card,1100 +50,45,bank_transfer,500 +51,46,bank_transfer,800 +52,47,credit_card,2200 +53,48,bank_transfer,300 +54,49,credit_card,600 +55,49,credit_card,900 +56,50,credit_card,2600 +57,51,credit_card,2900 +58,51,credit_card,100 +59,52,bank_transfer,1500 +60,53,credit_card,300 +61,54,credit_card,1800 +62,54,bank_transfer,1100 +63,55,credit_card,2900 +64,56,credit_card,400 +65,57,bank_transfer,200 +66,58,coupon,1800 +67,58,gift_card,600 +68,59,gift_card,2800 +69,60,credit_card,400 +70,61,bank_transfer,1600 +71,62,gift_card,1400 +72,63,credit_card,2900 +73,64,bank_transfer,2600 +74,65,credit_card,0 +75,66,credit_card,2800 +76,67,bank_transfer,400 +77,67,credit_card,1900 +78,68,credit_card,1600 +79,69,credit_card,1900 +80,70,credit_card,2600 +81,71,credit_card,500 +82,72,credit_card,2900 +83,73,bank_transfer,300 +84,74,credit_card,3000 +85,75,credit_card,1900 +86,76,coupon,200 +87,77,credit_card,0 +88,77,bank_transfer,1900 +89,78,bank_transfer,2600 +90,79,credit_card,1800 +91,79,credit_card,900 +92,80,gift_card,300 +93,81,coupon,200 +94,82,credit_card,800 +95,83,credit_card,100 +96,84,bank_transfer,2500 +97,85,bank_transfer,1700 +98,86,coupon,2300 +99,87,gift_card,3000 +100,87,credit_card,2600 +101,88,credit_card,2900 +102,89,bank_transfer,2200 +103,90,bank_transfer,200 +104,91,credit_card,1900 +105,92,bank_transfer,1500 +106,92,coupon,200 +107,93,gift_card,2600 +108,94,coupon,700 +109,95,coupon,2400 +110,96,gift_card,1700 +111,97,bank_transfer,1400 +112,98,bank_transfer,1000 +113,99,credit_card,2400 diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/test_dbt_project.py b/python_modules/libraries/dagster-components/dagster_components_tests/test_dbt_project.py deleted file mode 100644 index 0d35354a2a0ea..0000000000000 --- a/python_modules/libraries/dagster-components/dagster_components_tests/test_dbt_project.py +++ /dev/null @@ -1,75 +0,0 @@ -import shutil -import tempfile -from contextlib import contextmanager -from pathlib import Path -from typing import Generator - -import pytest -from dagster import AssetKey -from dagster_components.core.component_decl_builder import DefsFileModel -from dagster_components.core.component_defs_builder import ( - YamlComponentDecl, - build_components_from_component_folder, - defs_from_components, -) -from dagster_components.impls.dbt_project import DbtProjectComponent -from dagster_dbt import DbtProject - -from dagster_components_tests.utils import assert_assets, get_asset_keys, script_load_context - -STUB_LOCATION_PATH = Path(__file__).parent / "stub_code_locations" / "dbt_project_location" -COMPONENT_RELPATH = "components/jaffle_shop_dbt" - -JAFFLE_SHOP_KEYS = { - AssetKey("customers"), - AssetKey("orders"), - AssetKey("raw_customers"), - AssetKey("raw_orders"), - AssetKey("raw_payments"), - AssetKey("stg_customers"), - AssetKey("stg_orders"), - AssetKey("stg_payments"), -} - - -@contextmanager -@pytest.fixture(scope="module") -def dbt_path() -> Generator[Path, None, None]: - with tempfile.TemporaryDirectory() as temp_dir: - shutil.copytree(STUB_LOCATION_PATH, temp_dir, dirs_exist_ok=True) - # make sure a manifest.json file is created - project = DbtProject(Path(temp_dir) / "components/jaffle_shop_dbt/jaffle_shop") - project.preparer.prepare(project) - yield Path(temp_dir) - - -def test_python_params(dbt_path: Path) -> None: - component = DbtProjectComponent.from_decl_node( - context=script_load_context(), - decl_node=YamlComponentDecl( - path=dbt_path / COMPONENT_RELPATH, - defs_file_model=DefsFileModel( - component_type="dbt_project", - component_params={"dbt": {"project_dir": "jaffle_shop"}}, - ), - ), - ) - assert get_asset_keys(component) == JAFFLE_SHOP_KEYS - - -def test_load_from_path(dbt_path: Path) -> None: - components = build_components_from_component_folder( - script_load_context(), dbt_path / "components" - ) - assert len(components) == 1 - assert get_asset_keys(components[0]) == JAFFLE_SHOP_KEYS - - assert_assets(components[0], len(JAFFLE_SHOP_KEYS)) - - defs = defs_from_components( - context=script_load_context(), - components=components, - resources={}, - ) - - assert defs.get_asset_graph().get_all_asset_keys() == JAFFLE_SHOP_KEYS diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_pipes_subprocess_script_collection.py b/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_pipes_subprocess_script_collection.py index 9f2d7176a28df..ae4206ef1fe58 100644 --- a/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_pipes_subprocess_script_collection.py +++ b/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_pipes_subprocess_script_collection.py @@ -1,14 +1,14 @@ from pathlib import Path from dagster import AssetKey -from dagster_components.core.component_decl_builder import DefsFileModel +from dagster_components.core.component_decl_builder import ComponentFileModel from dagster_components.core.component_defs_builder import ( YamlComponentDecl, build_components_from_component_folder, build_defs_from_component_path, defs_from_components, ) -from dagster_components.impls.pipes_subprocess_script_collection import ( +from dagster_components.lib.pipes_subprocess_script_collection import ( PipesSubprocessScriptCollection, ) @@ -29,9 +29,9 @@ def test_python_params() -> None: load_context=script_load_context(), component_decl=YamlComponentDecl( path=LOCATION_PATH / "components" / "scripts", - defs_file_model=DefsFileModel( - component_type="pipes_subprocess_script_collection", - component_params={ + component_file_model=ComponentFileModel( + type="pipes_subprocess_script_collection", + params={ "scripts": [ { "path": "script_one.py", diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_registered_component.py b/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_registered_component.py new file mode 100644 index 0000000000000..bf7fddb6423f4 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_registered_component.py @@ -0,0 +1,32 @@ +from dagster_components import Component, component +from dagster_components.core.component import get_component_name, is_registered_component + + +def test_registered_component_with_default_name() -> None: + @component + class RegisteredComponent(Component): ... + + assert is_registered_component(RegisteredComponent) + assert get_component_name(RegisteredComponent) == "registered_component" + + +def test_registered_component_with_default_name_and_parens() -> None: + @component() + class RegisteredComponent(Component): ... + + assert is_registered_component(RegisteredComponent) + assert get_component_name(RegisteredComponent) == "registered_component" + + +def test_registered_component_with_explicit_kwarg_name() -> None: + @component(name="explicit_name") + class RegisteredComponent(Component): ... + + assert is_registered_component(RegisteredComponent) + assert get_component_name(RegisteredComponent) == "explicit_name" + + +def test_unregistered_component() -> None: + class UnregisteredComponent(Component): ... + + assert not is_registered_component(UnregisteredComponent) diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_registry.py b/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_registry.py new file mode 100644 index 0000000000000..0c52535b44096 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_registry.py @@ -0,0 +1,119 @@ +import os +import subprocess +import sys +from pathlib import Path + +from dagster_components import ComponentRegistry + + +def test_components_from_dagster(): + registry = ComponentRegistry.from_entry_point_discovery() + assert registry.has("dagster_components.dbt_project") + assert registry.has("dagster_components.sling_replication") + assert registry.has("dagster_components.pipes_subprocess_script_collection") + + +def _find_repo_root(): + current = Path(__file__).parent + while not (current / ".git").exists(): + if current == Path("/"): + raise Exception("Could not find the repository root.") + current = current.parent + return current + + +repo_root = _find_repo_root() + +# Our pyproject.toml installs local dagster components +PYPROJECT_TOML = f""" +[build-system] +requires = ["setuptools", "wheel"] +build-backend = "setuptools.build_meta" + +[project] +name = "dagster-foo" +version = "0.1.0" +description = "A simple example package" +authors = [ + {{ name = "Your Name", email = "your.email@example.com" }} +] +dependencies = [ + "dagster", + "dagster-components", + "dagster-dbt", + "dagster-embedded-elt", +] + +[tool.uv.sources] +dagster = {{ path = "{repo_root}/python_modules/dagster" }} +dagster-pipes = {{ path = "{repo_root}/python_modules/dagster-pipes" }} +dagster-components = {{ path = "{repo_root}/python_modules/libraries/dagster-components" }} +dagster-dbt = {{ path = "{repo_root}/python_modules/libraries/dagster-dbt" }} +dagster-embedded-elt = {{ path = "{repo_root}/python_modules/libraries/dagster-embedded-elt" }} + +[project.entry-points] +"dagster.components" = {{ dagster_foo = "dagster_foo.lib"}} +""" + +TEST_COMPONENT_1 = """ +from dagster_components import Component, component + +@component(name="test_component_1") +class TestComponent1(Component): + pass +""" + +TEST_COMPONENT_2 = """ +from dagster_components import Component, component + +@component(name="test_component_2") +class TestComponent2(Component): + pass +""" + +COMPONENT_PRINT_SCRIPT = """ +from dagster_components import ComponentRegistry + +registry = ComponentRegistry.from_entry_point_discovery() +for component_name in list(registry.keys()): + print(component_name) +""" + + +def test_components_from_third_party_lib(tmpdir): + with tmpdir.as_cwd(): + # Create test package that defines some components + os.makedirs("dagster-foo") + with open("dagster-foo/pyproject.toml", "w") as f: + f.write(PYPROJECT_TOML) + + os.makedirs("dagster-foo/dagster_foo/lib/sub") + + with open("dagster-foo/dagster_foo/lib/__init__.py", "w") as f: + f.write(TEST_COMPONENT_1) + + with open("dagster-foo/dagster_foo/lib/sub/__init__.py", "w") as f: + f.write(TEST_COMPONENT_2) + + # Create venv + venv_dir = Path(".venv") + subprocess.check_call(["uv", "venv", str(venv_dir)]) + python_executable = ( + venv_dir + / ("Scripts" if sys.platform == "win32" else "bin") + / ("python.exe" if sys.platform == "win32" else "python") + ) + + # Script to print components + with open("print_components.py", "w") as f: + f.write(COMPONENT_PRINT_SCRIPT) + + # subprocess.check_call([pip_executable, "install", "-e", "dagster-foo"]) + subprocess.check_call( + ["uv", "pip", "install", "--python", str(python_executable), "-e", "dagster-foo"] + ) + result = subprocess.run( + [python_executable, "print_components.py"], capture_output=True, text=True, check=False + ) + assert "dagster_foo.test_component_1" in result.stdout + assert "dagster_foo.test_component_2" in result.stdout diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/utils.py b/python_modules/libraries/dagster-components/dagster_components_tests/utils.py index a6655490e423e..51bce9a14ea32 100644 --- a/python_modules/libraries/dagster-components/dagster_components_tests/utils.py +++ b/python_modules/libraries/dagster-components/dagster_components_tests/utils.py @@ -1,10 +1,9 @@ from dagster import AssetKey, DagsterInstance -from dagster_components import __component_registry__ from dagster_components.core.component import Component, ComponentLoadContext, ComponentRegistry def registry() -> ComponentRegistry: - return ComponentRegistry(__component_registry__) + return ComponentRegistry.from_entry_point_discovery() def script_load_context() -> ComponentLoadContext: diff --git a/python_modules/libraries/dagster-components/setup.py b/python_modules/libraries/dagster-components/setup.py index a0aa10e3a9f4b..903e3fa1c9f7b 100644 --- a/python_modules/libraries/dagster-components/setup.py +++ b/python_modules/libraries/dagster-components/setup.py @@ -37,12 +37,16 @@ def get_version() -> str: packages=find_packages(exclude=["dagster_components_tests*", "examples*"]), install_requires=[ f"dagster{pin}", + "tomli", ], zip_safe=False, entry_points={ "console_scripts": [ - "dg = dagster_components.cli:main", - ] + "dagster-components = dagster_components.cli:main", + ], + "dagster.components": [ + "dagster_components = dagster_components.lib", + ], }, extras_require={ "sling": ["dagster-embedded-elt"], diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt/dbt_project.py b/python_modules/libraries/dagster-dbt/dagster_dbt/dbt_project.py index 11fc7d1ad7435..7178259594395 100644 --- a/python_modules/libraries/dagster-dbt/dagster_dbt/dbt_project.py +++ b/python_modules/libraries/dagster-dbt/dagster_dbt/dbt_project.py @@ -199,6 +199,7 @@ def get_env(): """ + name: str project_dir: Path target_path: Path target: Optional[str] @@ -250,6 +251,7 @@ def __new__( return super().__new__( cls, + name=dbt_project_yml["name"], project_dir=project_dir, target_path=target_path, target=target, diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt/include/pyproject.toml.jinja b/python_modules/libraries/dagster-dbt/dagster_dbt/include/pyproject.toml.jinja index 99e5f74f20e7e..8dbdd112518e2 100644 --- a/python_modules/libraries/dagster-dbt/dagster_dbt/include/pyproject.toml.jinja +++ b/python_modules/libraries/dagster-dbt/dagster_dbt/include/pyproject.toml.jinja @@ -1,3 +1,23 @@ +[project] +name = "{{ project_name }}" +version = "0.1.0" +description = "Add your description here" +readme = "README.md" +requires-python = ">=3.9,<3.13" +dependencies = [ + "dagster", + "dagster-cloud", + "dagster-dbt", + {%- for dbt_adapter in dbt_adapter_packages %} + "{{ dbt_adapter }}<{{ dbt_core_version_upper_bound }}", + {%- endfor %} +] + +[project.optional-dependencies] +dev = [ + "dagster-webserver", +] + [build-system] requires = ["setuptools"] build-backend = "setuptools.build_meta" diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_decorator.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_decorator.py index 9c276d552b070..12989283941f9 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_decorator.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_decorator.py @@ -33,6 +33,7 @@ def fivetran_assets( Sync the tables of a Fivetran connector: .. code-block:: python + from dagster_fivetran import FivetranWorkspace, fivetran_assets import dagster as dg @@ -60,6 +61,7 @@ def fivetran_connector_assets(context: dg.AssetExecutionContext, fivetran: Fivet Sync the tables of a Fivetran connector with a custom translator: .. code-block:: python + from dagster_fivetran import ( DagsterFivetranTranslator, FivetranConnectorTableProps, diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_defs.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_defs.py index 74d5752ea72c2..f746c6b3728ea 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_defs.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_defs.py @@ -757,6 +757,7 @@ def build_fivetran_assets_definitions( Sync the tables of a Fivetran connector: .. code-block:: python + from dagster_fivetran import FivetranWorkspace, build_fivetran_assets_definitions import dagster as dg @@ -777,6 +778,7 @@ def build_fivetran_assets_definitions( Sync the tables of a Fivetran connector with a custom translator: .. code-block:: python + from dagster_fivetran import ( DagsterFivetranTranslator, FivetranConnectorTableProps, diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran/resources.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran/resources.py index a2c82215d0e46..aa595b58768e4 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran/resources.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran/resources.py @@ -630,6 +630,24 @@ def update_schedule_type_for_connector( method="PATCH", endpoint=connector_id, data=json.dumps({"schedule_type": schedule_type}) ) + def get_columns_config_for_table( + self, connector_id: str, schema_name: str, table_name: str + ) -> Mapping[str, Any]: + """Fetches the source table columns config for a given table from the Fivetran API. + + Args: + connector_id (str): The Fivetran Connector ID. + schema_name (str): The Fivetran Schema name. + table_name (str): The Fivetran Table name. + + Returns: + Dict[str, Any]: Parsed json data from the response to this request. + """ + return self._make_connector_request( + method="GET", + endpoint=f"{connector_id}/schemas/{schema_name}/tables/{table_name}/columns", + ) + def start_sync(self, connector_id: str) -> None: """Initiates a sync of a Fivetran connector. @@ -856,7 +874,6 @@ def fetch_fivetran_workspace_data( self, ) -> FivetranWorkspaceData: """Retrieves all Fivetran content from the workspace and returns it as a FivetranWorkspaceData object. - Future work will cache this data to avoid repeated calls to the Fivetran API. Returns: FivetranWorkspaceData: A snapshot of the Fivetran workspace's content. @@ -1064,6 +1081,7 @@ def load_fivetran_asset_specs( Loading the asset specs for a given Fivetran workspace: .. code-block:: python + from dagster_fivetran import FivetranWorkspace, load_fivetran_asset_specs import dagster as dg diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran/translator.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran/translator.py index e19e422235d6f..9d36533dc67fb 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran/translator.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran/translator.py @@ -153,7 +153,7 @@ class FivetranTable: enabled: bool name_in_destination: str - # We keep the raw data for columns to add it as `column_info in the metadata. + # We keep the raw data for columns to add it as `column_info` in the metadata. columns: Optional[Mapping[str, Any]] @classmethod @@ -259,7 +259,7 @@ def namespace(cls) -> str: class DagsterFivetranTranslator: """Translator class which converts a `FivetranConnectorTableProps` object into AssetSpecs. - Subclass this class to implement custom logic for each type of Fivetran content. + Subclass this class to implement custom logic on how to translate Fivetran content into asset spec. """ def get_asset_spec(self, props: FivetranConnectorTableProps) -> AssetSpec: diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/conftest.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/conftest.py index 3d8c5db724e0c..76df67e6416e3 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/conftest.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/conftest.py @@ -18,6 +18,10 @@ TEST_API_SECRET = "test_api_secret" TEST_ANOTHER_ACCOUNT_ID = "test_another_account_id" +TEST_SCHEMA_NAME = "schema_name_in_destination_1" +TEST_TABLE_NAME = "table_name_in_destination_1" +TEST_ANOTHER_TABLE_NAME = "another_table_name_in_destination_1" + # Taken from Fivetran API documentation # https://fivetran.com/docs/rest-api/api-reference/groups/list-all-groups SAMPLE_GROUPS = { @@ -401,16 +405,47 @@ def get_sample_schema_config_for_connector(table_name: str) -> Mapping[str, Any] SAMPLE_SCHEMA_CONFIG_FOR_CONNECTOR = get_sample_schema_config_for_connector( - table_name="table_name_in_destination_1" + table_name=TEST_TABLE_NAME ) # We change the name of the original example to test the sync and poll materialization method ALTERED_SAMPLE_SCHEMA_CONFIG_FOR_CONNECTOR = get_sample_schema_config_for_connector( - table_name="another_table_name_in_destination_1" + table_name=TEST_ANOTHER_TABLE_NAME ) SAMPLE_SUCCESS_MESSAGE = {"code": "Success", "message": "Operation performed."} +SAMPLE_SOURCE_TABLE_COLUMNS_CONFIG = { + "code": "Success", + "message": "Operation performed.", + "data": { + "columns": { + "property1": { + "name_in_destination": "column_name_in_destination_1", + "enabled": True, + "hashed": False, + "enabled_patch_settings": { + "allowed": False, + "reason": "...", + "reason_code": "SYSTEM_COLUMN", + }, + "is_primary_key": True, + }, + "property2": { + "name_in_destination": "column_name_in_destination_2", + "enabled": True, + "hashed": False, + "enabled_patch_settings": { + "allowed": False, + "reason": "...", + "reason_code": "SYSTEM_COLUMN", + }, + "is_primary_key": True, + }, + } + }, +} + def get_fivetran_connector_api_url(connector_id: str) -> str: return ( @@ -480,9 +515,10 @@ def all_api_mocks_fixture( group_id: str, fetch_workspace_data_api_mocks: responses.RequestsMock, ) -> Iterator[responses.RequestsMock]: + test_connector_api_url = get_fivetran_connector_api_url(connector_id) fetch_workspace_data_api_mocks.add( method=responses.GET, - url=get_fivetran_connector_api_url(connector_id), + url=test_connector_api_url, json=get_sample_connection_details( succeeded_at=TEST_MAX_TIME_STR, failed_at=TEST_PREVIOUS_MAX_TIME_STR ), @@ -490,7 +526,7 @@ def all_api_mocks_fixture( ) fetch_workspace_data_api_mocks.add( method=responses.PATCH, - url=get_fivetran_connector_api_url(connector_id), + url=test_connector_api_url, json=get_sample_connection_details( succeeded_at=TEST_MAX_TIME_STR, failed_at=TEST_PREVIOUS_MAX_TIME_STR ), @@ -498,22 +534,28 @@ def all_api_mocks_fixture( ) fetch_workspace_data_api_mocks.add( method=responses.POST, - url=f"{get_fivetran_connector_api_url(connector_id)}/force", + url=f"{test_connector_api_url}/force", json=SAMPLE_SUCCESS_MESSAGE, status=200, ) fetch_workspace_data_api_mocks.add( method=responses.POST, - url=f"{get_fivetran_connector_api_url(connector_id)}/resync", + url=f"{test_connector_api_url}/resync", json=SAMPLE_SUCCESS_MESSAGE, status=200, ) fetch_workspace_data_api_mocks.add( method=responses.POST, - url=f"{get_fivetran_connector_api_url(connector_id)}/schemas/tables/resync", + url=f"{test_connector_api_url}/schemas/tables/resync", json=SAMPLE_SUCCESS_MESSAGE, status=200, ) + fetch_workspace_data_api_mocks.add( + method=responses.GET, + url=f"{test_connector_api_url}/schemas/{TEST_SCHEMA_NAME}/tables/{TEST_TABLE_NAME}/columns", + json=SAMPLE_SOURCE_TABLE_COLUMNS_CONFIG, + status=200, + ) yield fetch_workspace_data_api_mocks diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/test_resources.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/test_resources.py index a371a44feec57..48da73164c512 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/test_resources.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/experimental/test_resources.py @@ -19,6 +19,8 @@ TEST_API_SECRET, TEST_MAX_TIME_STR, TEST_PREVIOUS_MAX_TIME_STR, + TEST_SCHEMA_NAME, + TEST_TABLE_NAME, get_fivetran_connector_api_url, get_sample_connection_details, ) @@ -58,6 +60,17 @@ def test_basic_resource_request( assert connector_id in all_api_mocks.calls[1].request.url assert all_api_mocks.calls[1].request.method == "PATCH" + # columns config calls + all_api_mocks.calls.reset() + client.get_columns_config_for_table( + connector_id=connector_id, schema_name=TEST_SCHEMA_NAME, table_name=TEST_TABLE_NAME + ) + assert len(all_api_mocks.calls) == 1 + assert ( + f"{connector_id}/schemas/{TEST_SCHEMA_NAME}/tables/{TEST_TABLE_NAME}/columns" + in all_api_mocks.calls[0].request.url + ) + # sync calls all_api_mocks.calls.reset() client.start_sync(connector_id=connector_id) diff --git a/python_modules/libraries/dg-cli/LICENSE b/python_modules/libraries/dg-cli/LICENSE new file mode 100644 index 0000000000000..1ba565405dd93 --- /dev/null +++ b/python_modules/libraries/dg-cli/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2023 Dagster Labs, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. \ No newline at end of file diff --git a/python_modules/libraries/dg-cli/MANIFEST.in b/python_modules/libraries/dg-cli/MANIFEST.in new file mode 100644 index 0000000000000..83ff507494d7e --- /dev/null +++ b/python_modules/libraries/dg-cli/MANIFEST.in @@ -0,0 +1,4 @@ +include README.md +include LICENSE +include dg_cli/py.typed +recursive-include dg_cli/templates * diff --git a/python_modules/libraries/dg-cli/README.md b/python_modules/libraries/dg-cli/README.md new file mode 100644 index 0000000000000..17251f6fdd754 --- /dev/null +++ b/python_modules/libraries/dg-cli/README.md @@ -0,0 +1,3 @@ +# dg-cli + +Experimental API for scaffolding Dagster projects. Includes the `dg` CLI tool. diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/__init__.py b/python_modules/libraries/dg-cli/dg_cli/__init__.py similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/__init__.py rename to python_modules/libraries/dg-cli/dg_cli/__init__.py diff --git a/python_modules/libraries/dg-cli/dg_cli/cli/__init__.py b/python_modules/libraries/dg-cli/dg_cli/cli/__init__.py new file mode 100644 index 0000000000000..0e49fea365c46 --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/cli/__init__.py @@ -0,0 +1,30 @@ +import click + +from dg_cli.cli.generate import generate_cli +from dg_cli.cli.list import list_cli +from dg_cli.version import __version__ + + +def create_dg_cli(): + commands = { + "generate": generate_cli, + "list": list_cli, + } + + @click.group( + commands=commands, + context_settings={"max_content_width": 120, "help_option_names": ["-h", "--help"]}, + ) + @click.version_option(__version__, "--version", "-v") + def group(): + """CLI tools for working with Dagster.""" + + return group + + +ENV_PREFIX = "DG_CLI" +cli = create_dg_cli() + + +def main(): + cli(auto_envvar_prefix=ENV_PREFIX) diff --git a/python_modules/libraries/dg-cli/dg_cli/cli/generate.py b/python_modules/libraries/dg-cli/dg_cli/cli/generate.py new file mode 100644 index 0000000000000..0f0da7bd0033d --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/cli/generate.py @@ -0,0 +1,128 @@ +import os +import sys +from pathlib import Path +from typing import Optional, Tuple + +import click + +from dg_cli.context import ( + CodeLocationProjectContext, + DeploymentProjectContext, + is_inside_code_location_project, + is_inside_deployment_project, +) +from dg_cli.generate import ( + generate_code_location, + generate_component_instance, + generate_component_type, + generate_deployment, +) + + +@click.group(name="generate") +def generate_cli() -> None: + """Commands for generating Dagster components and related entities.""" + + +@generate_cli.command(name="deployment") +@click.argument("path", type=str) +def generate_deployment_command(path: str) -> None: + """Generate a Dagster deployment instance.""" + dir_abspath = os.path.abspath(path) + if os.path.exists(dir_abspath): + click.echo( + click.style(f"A file or directory at {dir_abspath} already exists. ", fg="red") + + "\nPlease delete the contents of this path or choose another location." + ) + sys.exit(1) + generate_deployment(path) + + +@generate_cli.command(name="code-location") +@click.argument("name", type=str) +@click.option("--use-editable-dagster", is_flag=True, default=False) +def generate_code_location_command(name: str, use_editable_dagster: bool) -> None: + """Generate a Dagster code location inside a component.""" + if is_inside_deployment_project(Path.cwd()): + context = DeploymentProjectContext.from_path(Path.cwd()) + if context.has_code_location(name): + click.echo(click.style(f"A code location named {name} already exists.", fg="red")) + sys.exit(1) + code_location_path = os.path.join(context.code_location_root_path, name) + else: + code_location_path = os.path.join(Path.cwd(), name) + + if use_editable_dagster: + if "DAGSTER_GIT_REPO_DIR" not in os.environ: + click.echo( + click.style( + "The `--use-editable-dagster` flag requires the `DAGSTER_GIT_REPO_DIR` environment variable to be set.", + fg="red", + ) + ) + sys.exit(1) + editable_dagster_root = os.environ["DAGSTER_GIT_REPO_DIR"] + else: + editable_dagster_root = None + + generate_code_location(code_location_path, editable_dagster_root) + + +@generate_cli.command(name="component-type") +@click.argument("name", type=str) +def generate_component_type_command(name: str) -> None: + """Generate a Dagster component instance.""" + if not is_inside_code_location_project(Path.cwd()): + click.echo( + click.style( + "This command must be run inside a Dagster code location project.", fg="red" + ) + ) + sys.exit(1) + context = CodeLocationProjectContext.from_path(Path.cwd()) + full_component_name = f"{context.name}.{name}" + if context.has_component_type(full_component_name): + click.echo(click.style(f"A component type named `{name}` already exists.", fg="red")) + sys.exit(1) + + generate_component_type(context.component_types_root_path, name) + + +@generate_cli.command(name="component") +@click.argument("component_type", type=str) +@click.argument("component_name", type=str) +@click.option("--json-params", type=str, default=None) +@click.argument("extra_args", nargs=-1, type=str) +def generate_component_command( + component_type: str, + component_name: str, + json_params: Optional[str], + extra_args: Tuple[str, ...], +) -> None: + if not is_inside_code_location_project(Path.cwd()): + click.echo( + click.style( + "This command must be run inside a Dagster code location project.", fg="red" + ) + ) + sys.exit(1) + + context = CodeLocationProjectContext.from_path(Path.cwd()) + if not context.has_component_type(component_type): + click.echo( + click.style(f"No component type `{component_type}` could be resolved.", fg="red") + ) + sys.exit(1) + elif context.has_component_instance(component_name): + click.echo( + click.style(f"A component instance named `{component_name}` already exists.", fg="red") + ) + sys.exit(1) + + generate_component_instance( + context.component_instances_root_path, + component_name, + component_type, + json_params, + extra_args, + ) diff --git a/python_modules/libraries/dg-cli/dg_cli/cli/list.py b/python_modules/libraries/dg-cli/dg_cli/cli/list.py new file mode 100644 index 0000000000000..c20e4e2467922 --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/cli/list.py @@ -0,0 +1,62 @@ +import sys +from pathlib import Path + +import click + +from dg_cli.context import ( + CodeLocationProjectContext, + DeploymentProjectContext, + is_inside_code_location_project, + is_inside_deployment_project, +) + + +@click.group(name="list") +def list_cli(): + """Commands for listing Dagster components and related entities.""" + + +@list_cli.command(name="code-locations") +def list_code_locations_command() -> None: + """List code locations in the current deployment.""" + if not is_inside_deployment_project(Path.cwd()): + click.echo( + click.style("This command must be run inside a Dagster deployment project.", fg="red") + ) + sys.exit(1) + + context = DeploymentProjectContext.from_path(Path.cwd()) + for code_location in context.list_code_locations(): + click.echo(code_location) + + +@list_cli.command(name="component-types") +def list_component_types_command() -> None: + """List registered Dagster components.""" + if not is_inside_code_location_project(Path.cwd()): + click.echo( + click.style( + "This command must be run inside a Dagster code location project.", fg="red" + ) + ) + sys.exit(1) + + context = CodeLocationProjectContext.from_path(Path.cwd()) + for component_type in context.list_component_types(): + click.echo(component_type) + + +@list_cli.command(name="components") +def list_components_command() -> None: + """List Dagster component instances in a code location.""" + if not is_inside_code_location_project(Path.cwd()): + click.echo( + click.style( + "This command must be run inside a Dagster code location project.", fg="red" + ) + ) + sys.exit(1) + + context = CodeLocationProjectContext.from_path(Path.cwd()) + for component_name in context.component_instances: + click.echo(component_name) diff --git a/python_modules/libraries/dg-cli/dg_cli/component.py b/python_modules/libraries/dg-cli/dg_cli/component.py new file mode 100644 index 0000000000000..129c4d6c388ad --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/component.py @@ -0,0 +1,40 @@ +import copy +from dataclasses import dataclass +from typing import Any, Dict, Iterable, Mapping + + +@dataclass +class RemoteComponentType: + name: str + + @property + def key(self) -> str: + return self.name + # return f"{self.package}.{self.name}" + + +class RemoteComponentRegistry: + @classmethod + def from_dict(cls, components: Dict[str, Mapping[str, Any]]) -> "RemoteComponentRegistry": + return RemoteComponentRegistry( + {key: RemoteComponentType(**value) for key, value in components.items()} + ) + + def __init__(self, components: Dict[str, RemoteComponentType]): + self._components: Dict[str, RemoteComponentType] = copy.copy(components) + + @staticmethod + def empty() -> "RemoteComponentRegistry": + return RemoteComponentRegistry({}) + + def has(self, name: str) -> bool: + return name in self._components + + def get(self, name: str) -> RemoteComponentType: + return self._components[name] + + def keys(self) -> Iterable[str]: + return self._components.keys() + + def __repr__(self) -> str: + return f"" diff --git a/python_modules/libraries/dg-cli/dg_cli/context.py b/python_modules/libraries/dg-cli/dg_cli/context.py new file mode 100644 index 0000000000000..3f651315cb63a --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/context.py @@ -0,0 +1,173 @@ +import json +import os +from pathlib import Path +from typing import Final, Iterable, Mapping, Optional, Sequence + +import tomli +from typing_extensions import Self + +from dg_cli.component import RemoteComponentRegistry, RemoteComponentType +from dg_cli.error import DgError +from dg_cli.utils import execute_code_location_command + + +def is_inside_deployment_project(path: Path) -> bool: + try: + _resolve_deployment_root_path(path) + return True + except DgError: + return False + + +def _resolve_deployment_root_path(path: Path) -> Path: + current_path = path.absolute() + while not _is_deployment_root(current_path): + current_path = current_path.parent + if str(current_path) == "/": + raise DgError("Cannot find deployment root") + return current_path + + +def is_inside_code_location_project(path: Path) -> bool: + try: + _resolve_code_location_root_path(path) + return True + except DgError: + return False + + +def _resolve_code_location_root_path(path: Path) -> Path: + current_path = path.absolute() + while not _is_code_location_root(current_path): + current_path = current_path.parent + if str(current_path) == "/": + raise DgError("Cannot find code location root") + return current_path + + +def _is_deployment_root(path: Path) -> bool: + return (path / "code_locations").exists() + + +def _is_code_location_root(path: Path) -> bool: + if (path / "pyproject.toml").exists(): + with open(path / "pyproject.toml") as f: + toml = tomli.loads(f.read()) + return bool(toml.get("tool", {}).get("dagster")) + return False + + +# Deployment +_DEPLOYMENT_CODE_LOCATIONS_DIR: Final = "code_locations" + +# Code location +_CODE_LOCATION_CUSTOM_COMPONENTS_DIR: Final = "lib" +_CODE_LOCATION_COMPONENT_INSTANCES_DIR: Final = "components" + + +class DeploymentProjectContext: + @classmethod + def from_path(cls, path: Path) -> Self: + return cls(root_path=_resolve_deployment_root_path(path)) + + def __init__(self, root_path: Path): + self._root_path = root_path + + @property + def deployment_root(self) -> Path: + return self._root_path + + @property + def code_location_root_path(self) -> Path: + return self._root_path / _DEPLOYMENT_CODE_LOCATIONS_DIR + + def has_code_location(self, name: str) -> bool: + return os.path.exists(os.path.join(self._root_path, "code_locations", name)) + + def list_code_locations(self) -> Iterable[str]: + return sorted(os.listdir(os.path.join(self._root_path, "code_locations"))) + + +class CodeLocationProjectContext: + _components_registry: Mapping[str, RemoteComponentType] = {} + + @classmethod + def from_path(cls, path: Path) -> Self: + root_path = _resolve_code_location_root_path(path) + raw_component_registry = execute_code_location_command( + root_path, ["list", "component-types"] + ) + component_registry = RemoteComponentRegistry.from_dict(json.loads(raw_component_registry)) + deployment_context = ( + DeploymentProjectContext.from_path(path) if is_inside_deployment_project(path) else None + ) + + return cls( + deployment_context=deployment_context, + root_path=root_path, + name=path.name, + component_registry=component_registry, + ) + + def __init__( + self, + deployment_context: Optional[DeploymentProjectContext], + root_path: Path, + name: str, + component_registry: "RemoteComponentRegistry", + ): + self._deployment_context = deployment_context + self._root_path = root_path + self._name = name + self._component_registry = component_registry + + @property + def name(self) -> str: + return self._name + + @property + def deployment_context(self) -> Optional[DeploymentProjectContext]: + return self._deployment_context + + @property + def component_types_root_path(self) -> str: + return os.path.join(self._root_path, self._name, _CODE_LOCATION_CUSTOM_COMPONENTS_DIR) + + @property + def component_types_root_module(self) -> str: + return f"{self._name}.{_CODE_LOCATION_CUSTOM_COMPONENTS_DIR}" + + @property + def component_registry(self) -> "RemoteComponentRegistry": + return self._component_registry + + def has_component_type(self, name: str) -> bool: + return self._component_registry.has(name) + + def get_component_type(self, name: str) -> RemoteComponentType: + if not self.has_component_type(name): + raise DgError(f"No component type named {name}") + return self._component_registry.get(name) + + def list_component_types(self) -> Sequence[str]: + return sorted(self._component_registry.keys()) + + def get_component_instance_path(self, name: str) -> str: + if name not in self.component_instances: + raise DgError(f"No component instance named {name}") + return os.path.join(self.component_instances_root_path, name) + + @property + def component_instances_root_path(self) -> str: + return os.path.join(self._root_path, self._name, _CODE_LOCATION_COMPONENT_INSTANCES_DIR) + + @property + def component_instances(self) -> Iterable[str]: + return os.listdir( + os.path.join(self._root_path, self._name, _CODE_LOCATION_COMPONENT_INSTANCES_DIR) + ) + + def has_component_instance(self, name: str) -> bool: + return os.path.exists( + os.path.join(self._root_path, self._name, _CODE_LOCATION_COMPONENT_INSTANCES_DIR, name) + ) diff --git a/python_modules/libraries/dg-cli/dg_cli/error.py b/python_modules/libraries/dg-cli/dg_cli/error.py new file mode 100644 index 0000000000000..a6cdf04a81995 --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/error.py @@ -0,0 +1,2 @@ +class DgError(Exception): + pass diff --git a/python_modules/libraries/dg-cli/dg_cli/generate.py b/python_modules/libraries/dg-cli/dg_cli/generate.py new file mode 100644 index 0000000000000..7176daade49fe --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/generate.py @@ -0,0 +1,106 @@ +import os +import textwrap +from pathlib import Path +from typing import Optional, Tuple + +import click + +from dg_cli.utils import ( + camelcase, + discover_git_root, + execute_code_location_command, + generate_subtree, +) + + +def generate_deployment(path: str) -> None: + click.echo(f"Creating a Dagster deployment at {path}.") + + generate_subtree( + path=path, + name_placeholder="DEPLOYMENT_NAME_PLACEHOLDER", + templates_path=os.path.join( + os.path.dirname(__file__), "templates", "DEPLOYMENT_NAME_PLACEHOLDER" + ), + ) + + +def generate_code_location(path: str, editable_dagster_root: Optional[str] = None) -> None: + click.echo(f"Creating a Dagster code location at {path}.") + + # Temporarily we always set an editable dagster root. This is needed while the packages are not + # published. + editable_dagster_root = ( + editable_dagster_root + or os.environ.get("DAGSTER_GIT_REPO_DIR") + or discover_git_root(Path(__file__)) + ) + + editable_dagster_uv_sources = textwrap.dedent(f""" + [tool.uv.sources] + dagster = {{ path = "{editable_dagster_root}/python_modules/dagster", editable = true }} + dagster-graphql = {{ path = "{editable_dagster_root}/python_modules/dagster-graphql", editable = true }} + dagster-pipes = {{ path = "{editable_dagster_root}/python_modules/dagster-pipes", editable = true }} + dagster-webserver = {{ path = "{editable_dagster_root}/python_modules/dagster-webserver", editable = true }} + dagster-components = {{ path = "{editable_dagster_root}/python_modules/libraries/dagster-components", editable = true }} + dagster-embedded-elt = {{ path = "{editable_dagster_root}/python_modules/libraries/dagster-embedded-elt", editable = true }} + dagster-dbt = {{ path = "{editable_dagster_root}/python_modules/libraries/dagster-dbt", editable = true }} + """) + + if editable_dagster_root: + uv_sources = editable_dagster_uv_sources + else: + uv_sources = editable_dagster_uv_sources + + generate_subtree( + path=path, + name_placeholder="CODE_LOCATION_NAME_PLACEHOLDER", + templates_path=os.path.join( + os.path.dirname(__file__), "templates", "CODE_LOCATION_NAME_PLACEHOLDER" + ), + uv_sources=uv_sources, + ) + + +def generate_component_type(root_path: str, name: str) -> None: + click.echo(f"Creating a Dagster component type at {root_path}/{name}.py.") + + generate_subtree( + path=root_path, + name_placeholder="COMPONENT_TYPE_NAME_PLACEHOLDER", + templates_path=os.path.join(os.path.dirname(__file__), "templates", "COMPONENT_TYPE"), + project_name=name, + component_type_class_name=camelcase(name), + component_type=name, + ) + + +def generate_component_instance( + root_path: str, + name: str, + component_type: str, + json_params: Optional[str], + extra_args: Tuple[str, ...], +) -> None: + click.echo(f"Creating a Dagster component instance at {root_path}/{name}.py.") + + component_instance_root_path = os.path.join(root_path, name) + generate_subtree( + path=component_instance_root_path, + name_placeholder="COMPONENT_INSTANCE_NAME_PLACEHOLDER", + templates_path=os.path.join( + os.path.dirname(__file__), "templates", "COMPONENT_INSTANCE_NAME_PLACEHOLDER" + ), + project_name=name, + component_type=component_type, + ) + + code_location_command = ( + "generate", + "component", + component_type, + name, + *([f"--json-params={json_params}"] if json_params else []), + *(["--", *extra_args] if extra_args else []), + ) + execute_code_location_command(Path(component_instance_root_path), code_location_command) diff --git a/python_modules/libraries/dagster-components/dagster_components/core/registry.py b/python_modules/libraries/dg-cli/dg_cli/py.typed similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/core/registry.py rename to python_modules/libraries/dg-cli/dg_cli/py.typed diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/lib/__init__.py b/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/__init__.py similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/lib/__init__.py rename to python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/__init__.py diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/DEPLOYMENT_NAME_PLACEHOLDER/code_locations/.gitkeep b/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/components/.gitkeep similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/DEPLOYMENT_NAME_PLACEHOLDER/code_locations/.gitkeep rename to python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/components/.gitkeep diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/definitions.py b/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/definitions.py similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/definitions.py rename to python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/definitions.py diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER_tests/__init__.py b/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/lib/__init__.py similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER_tests/__init__.py rename to python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER/lib/__init__.py diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/DEPLOYMENT_NAME_PLACEHOLDER/.github/workflows/dagster-cloud-deploy.yaml.jinja b/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER_tests/__init__.py similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/DEPLOYMENT_NAME_PLACEHOLDER/.github/workflows/dagster-cloud-deploy.yaml.jinja rename to python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/CODE_LOCATION_NAME_PLACEHOLDER_tests/__init__.py diff --git a/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/pyproject.toml.jinja b/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/pyproject.toml.jinja new file mode 100644 index 0000000000000..688e8e80ab91d --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/templates/CODE_LOCATION_NAME_PLACEHOLDER/pyproject.toml.jinja @@ -0,0 +1,35 @@ +[project] +name = "{{ project_name }}" +requires-python = ">=3.9,<3.13" +version = "0.1.0" +dependencies = [ + "dagster", + "dagster-graphql", + "dagster-pipes", + "dagster-webserver", + "dagster-components[sling,dbt]", + "dagster-embedded-elt", + "dagster-dbt", + "sling-mac-arm64", +] + +[project.optional-dependencies] +dev = [ + "dagster-webserver", +] + +[project.entry-points] +"dagster.components" = { {{ project_name }} = "{{ project_name }}.lib"} + +[build-system] +requires = ["setuptools"] +build-backend = "setuptools.build_meta" + +[tool.dagster] +module_name = "{{ project_name }}.definitions" +project_name = "{{ project_name }}" + +[tool.setuptools.packages.find] +exclude=["{{ project_name }}_tests"] + +{{ uv_sources }} diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/COMPONENT_TYPE/COMPONENT_TYPE_NAME_PLACEHOLDER.py.jinja b/python_modules/libraries/dg-cli/dg_cli/templates/COMPONENT_TYPE/COMPONENT_TYPE_NAME_PLACEHOLDER.py.jinja similarity index 89% rename from python_modules/libraries/dagster-components/dagster_components/templates/COMPONENT_TYPE/COMPONENT_TYPE_NAME_PLACEHOLDER.py.jinja rename to python_modules/libraries/dg-cli/dg_cli/templates/COMPONENT_TYPE/COMPONENT_TYPE_NAME_PLACEHOLDER.py.jinja index a303fd0c12f76..9ac7ac51684c0 100644 --- a/python_modules/libraries/dagster-components/dagster_components/templates/COMPONENT_TYPE/COMPONENT_TYPE_NAME_PLACEHOLDER.py.jinja +++ b/python_modules/libraries/dg-cli/dg_cli/templates/COMPONENT_TYPE/COMPONENT_TYPE_NAME_PLACEHOLDER.py.jinja @@ -4,8 +4,10 @@ from dagster_components import ( ComponentRegistry, ComponentLoadContext, build_defs_from_toplevel_components_folder, + component, ) +@component(name="{{ name }}") class {{ component_type_class_name }}(Component): @classmethod diff --git a/python_modules/libraries/dagster-components/dagster_components/templates/DEPLOYMENT_NAME_PLACEHOLDER/dagster_cloud.yaml.jinja b/python_modules/libraries/dg-cli/dg_cli/templates/DEPLOYMENT_NAME_PLACEHOLDER/.github/workflows/dagster-cloud-deploy.yaml.jinja similarity index 100% rename from python_modules/libraries/dagster-components/dagster_components/templates/DEPLOYMENT_NAME_PLACEHOLDER/dagster_cloud.yaml.jinja rename to python_modules/libraries/dg-cli/dg_cli/templates/DEPLOYMENT_NAME_PLACEHOLDER/.github/workflows/dagster-cloud-deploy.yaml.jinja diff --git a/python_modules/libraries/dg-cli/dg_cli/templates/DEPLOYMENT_NAME_PLACEHOLDER/code_locations/.gitkeep b/python_modules/libraries/dg-cli/dg_cli/templates/DEPLOYMENT_NAME_PLACEHOLDER/code_locations/.gitkeep new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python_modules/libraries/dg-cli/dg_cli/templates/DEPLOYMENT_NAME_PLACEHOLDER/dagster_cloud.yaml.jinja b/python_modules/libraries/dg-cli/dg_cli/templates/DEPLOYMENT_NAME_PLACEHOLDER/dagster_cloud.yaml.jinja new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python_modules/libraries/dg-cli/dg_cli/utils.py b/python_modules/libraries/dg-cli/dg_cli/utils.py new file mode 100644 index 0000000000000..749cd69a26c45 --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli/utils.py @@ -0,0 +1,161 @@ +import contextlib +import os +import posixpath +import re +import subprocess +from pathlib import Path +from typing import Any, Final, Iterator, List, Optional, Sequence, Union + +import click +import jinja2 + +from dg_cli.version import __version__ as dagster_version + +_CODE_LOCATION_COMMAND_PREFIX: Final = ["uv", "run", "dagster-components"] + + +def discover_git_root(path: Path) -> str: + while path != path.parent: + if (path / ".git").exists(): + return str(path) + path = path.parent + raise ValueError("Could not find git root") + + +def execute_code_location_command(path: Path, cmd: Sequence[str]) -> str: + with pushd(path): + full_cmd = [*_CODE_LOCATION_COMMAND_PREFIX, *cmd] + result = subprocess.run(full_cmd, stdout=subprocess.PIPE, check=False) + return result.stdout.decode("utf-8") + + +@contextlib.contextmanager +def pushd(path: Union[str, Path]) -> Iterator[None]: + old_cwd = os.getcwd() + os.chdir(path) + try: + yield + finally: + os.chdir(old_cwd) + + +# Adapted from https://github.com/okunishinishi/python-stringcase/blob/master/stringcase.py +def camelcase(string: str) -> str: + string = re.sub(r"^[\-_\.]", "", str(string)) + if not string: + return string + return str(string[0]).upper() + re.sub( + r"[\-_\.\s]([a-z])", lambda matched: str(matched.group(1)).upper(), string[1:] + ) + + +def snakecase(string: str) -> str: + # Add an underscore before capital letters and lower the case + string = re.sub(r"(? Iterator[None]: @contextmanager -def isolated_example_code_location_bar(runner: CliRunner) -> Iterator[None]: - with isolated_example_deployment_foo(runner), clean_module_cache("bar"): - runner.invoke(generate_code_location_command, ["bar"]) - with pushd("code_locations/bar"): - yield +def isolated_example_code_location_bar( + runner: CliRunner, in_deployment: bool = True +) -> Iterator[None]: + if in_deployment: + with isolated_example_deployment_foo(runner), clean_module_cache("bar"): + runner.invoke(generate_code_location_command, ["bar"]) + with pushd("code_locations/bar"): + yield + else: + with runner.isolated_filesystem(), clean_module_cache("bar"): + runner.invoke(generate_code_location_command, ["bar"]) + with pushd("bar"): + yield @contextmanager -def isolated_example_code_location_bar_with_component_type_baz(runner: CliRunner) -> Iterator[None]: - with isolated_example_code_location_bar(runner): +def isolated_example_code_location_bar_with_component_type_baz( + runner: CliRunner, in_deployment: bool = True +) -> Iterator[None]: + with isolated_example_code_location_bar(runner, in_deployment): with open("bar/lib/baz.py", "w") as f: component_type_source = textwrap.dedent( inspect.getsource(_example_component_type_baz).split("\n", 1)[1] @@ -115,7 +130,7 @@ def test_generate_deployment_command_already_exists_fails() -> None: assert "already exists" in result.output -def test_generate_code_location_success() -> None: +def test_generate_code_location_inside_deployment_success() -> None: runner = CliRunner() with isolated_example_deployment_foo(runner): result = runner.invoke(generate_code_location_command, ["bar"]) @@ -127,13 +142,63 @@ def test_generate_code_location_success() -> None: assert Path("code_locations/bar/bar_tests").exists() assert Path("code_locations/bar/pyproject.toml").exists() + # Commented out because we are always adding sources right now + # with open("code_locations/bar/pyproject.toml") as f: + # toml = tomli.loads(f.read()) + # + # # No tool.uv.sources added without --use-editable-dagster + # assert "uv" not in toml["tool"] + -def test_generate_code_location_outside_deployment_fails() -> None: +def test_generate_code_location_outside_deployment_success() -> None: runner = CliRunner() with runner.isolated_filesystem(): result = runner.invoke(generate_code_location_command, ["bar"]) - assert result.exit_code != 0 - assert "must be run inside a Dagster deployment project" in result.output + assert result.exit_code == 0 + assert Path("bar").exists() + assert Path("bar/bar").exists() + assert Path("bar/bar/lib").exists() + assert Path("bar/bar/components").exists() + assert Path("bar/bar_tests").exists() + assert Path("bar/pyproject.toml").exists() + + +def _find_git_root(): + current = Path.cwd() + while current != current.parent: + if (current / ".git").exists(): + return current + current = current.parent + raise Exception("Could not find git root") + + +def test_generate_code_location_editable_dagster_success(monkeypatch) -> None: + runner = CliRunner() + dagster_git_repo_dir = _find_git_root() + monkeypatch.setenv("DAGSTER_GIT_REPO_DIR", dagster_git_repo_dir) + with isolated_example_deployment_foo(runner): + result = runner.invoke(generate_code_location_command, ["--use-editable-dagster", "bar"]) + assert result.exit_code == 0 + assert Path("code_locations/bar").exists() + assert Path("code_locations/bar/pyproject.toml").exists() + with open("code_locations/bar/pyproject.toml") as f: + toml = tomli.loads(f.read()) + assert toml["tool"]["uv"]["sources"]["dagster"] == { + "path": f"{dagster_git_repo_dir}/python_modules/dagster", + "editable": True, + } + assert toml["tool"]["uv"]["sources"]["dagster-pipes"] == { + "path": f"{dagster_git_repo_dir}/python_modules/dagster-pipes", + "editable": True, + } + assert toml["tool"]["uv"]["sources"]["dagster-webserver"] == { + "path": f"{dagster_git_repo_dir}/python_modules/dagster-webserver", + "editable": True, + } + assert toml["tool"]["uv"]["sources"]["dagster-components"] == { + "path": f"{dagster_git_repo_dir}/python_modules/libraries/dagster-components", + "editable": True, + } def test_generate_code_location_already_exists_fails() -> None: @@ -146,15 +211,15 @@ def test_generate_code_location_already_exists_fails() -> None: assert "already exists" in result.output -def test_generate_component_type_success() -> None: +@pytest.mark.parametrize("in_deployment", [True, False]) +def test_generate_component_type_success(in_deployment: bool) -> None: runner = CliRunner() - with isolated_example_code_location_bar(runner): + with isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke(generate_component_type_command, ["baz"]) assert result.exit_code == 0 assert Path("bar/lib/baz.py").exists() - _assert_module_imports("bar.lib.baz") - context = CodeLocationProjectContext.from_path(Path.cwd(), ComponentRegistry.empty()) - assert context.has_component_type("baz") + context = CodeLocationProjectContext.from_path(Path.cwd()) + assert context.has_component_type("bar.baz") def test_generate_component_type_outside_code_location_fails() -> None: @@ -165,9 +230,10 @@ def test_generate_component_type_outside_code_location_fails() -> None: assert "must be run inside a Dagster code location project" in result.output -def test_generate_component_type_already_exists_fails() -> None: +@pytest.mark.parametrize("in_deployment", [True, False]) +def test_generate_component_type_already_exists_fails(in_deployment: bool) -> None: runner = CliRunner() - with isolated_example_code_location_bar(runner): + with isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke(generate_component_type_command, ["baz"]) assert result.exit_code == 0 result = runner.invoke(generate_component_type_command, ["baz"]) @@ -175,46 +241,79 @@ def test_generate_component_type_already_exists_fails() -> None: assert "already exists" in result.output -def test_generate_component_success() -> None: +@pytest.mark.parametrize("in_deployment", [True, False]) +def test_generate_component_success(in_deployment: bool) -> None: runner = CliRunner() - _ensure_cwd_on_sys_path() - with isolated_example_code_location_bar_with_component_type_baz(runner): - result = runner.invoke(generate_component_command, ["baz", "qux"]) + with isolated_example_code_location_bar_with_component_type_baz(runner, in_deployment): + result = runner.invoke(generate_component_command, ["bar.baz", "qux"]) assert result.exit_code == 0 assert Path("bar/components/qux").exists() assert Path("bar/components/qux/sample.py").exists() + component_yaml_path = Path("bar/components/qux/component.yaml") + assert component_yaml_path.exists() + assert "type: bar.baz" in component_yaml_path.read_text() def test_generate_component_outside_code_location_fails() -> None: runner = CliRunner() with isolated_example_deployment_foo(runner): - result = runner.invoke(generate_component_command, ["baz", "qux"]) + result = runner.invoke(generate_component_command, ["bar.baz", "qux"]) assert result.exit_code != 0 assert "must be run inside a Dagster code location project" in result.output -def test_generate_component_already_exists_fails() -> None: +@pytest.mark.parametrize("in_deployment", [True, False]) +def test_generate_component_already_exists_fails(in_deployment: bool) -> None: runner = CliRunner() - _ensure_cwd_on_sys_path() - with isolated_example_code_location_bar_with_component_type_baz(runner): - result = runner.invoke(generate_component_command, ["baz", "qux"]) + with isolated_example_code_location_bar_with_component_type_baz(runner, in_deployment): + result = runner.invoke(generate_component_command, ["bar.baz", "qux"]) assert result.exit_code == 0 - result = runner.invoke(generate_component_command, ["baz", "qux"]) + result = runner.invoke(generate_component_command, ["bar.baz", "qux"]) assert result.exit_code != 0 assert "already exists" in result.output -def test_generate_global_component_instance() -> None: +def test_generate_sling_replication_instance() -> None: runner = CliRunner() with isolated_example_code_location_bar(runner): - result = runner.invoke(generate_component_command, ["sling_replication", "file_ingest"]) + result = runner.invoke( + generate_component_command, ["dagster_components.sling_replication", "file_ingest"] + ) assert result.exit_code == 0 assert Path("bar/components/file_ingest").exists() - defs_path = Path("bar/components/file_ingest/defs.yml") - assert defs_path.exists() - assert "component_type: sling_replication" in defs_path.read_text() + component_yaml_path = Path("bar/components/file_ingest/component.yaml") + assert component_yaml_path.exists() + assert "type: dagster_components.sling_replication" in component_yaml_path.read_text() replication_path = Path("bar/components/file_ingest/replication.yaml") assert replication_path.exists() assert "source: " in replication_path.read_text() + + +dbt_project_path = "../stub_code_locations/dbt_project_location/components/jaffle_shop" + + +@pytest.mark.parametrize( + "params", + [ + ["--json-params", json.dumps({"project_path": str(dbt_project_path)})], + ["--", "--project-path", dbt_project_path], + ], +) +def test_generate_dbt_project_instance(params) -> None: + runner = CliRunner() + with isolated_example_code_location_bar(runner): + result = runner.invoke( + generate_component_command, ["dagster_components.dbt_project", "my_project", *params] + ) + assert result.exit_code == 0 + assert Path("bar/components/my_project").exists() + + component_yaml_path = Path("bar/components/my_project/component.yaml") + assert component_yaml_path.exists() + assert "type: dagster_components.dbt_project" in component_yaml_path.read_text() + assert ( + "stub_code_locations/dbt_project_location/components/jaffle_shop" + in component_yaml_path.read_text() + ) diff --git a/python_modules/libraries/dg-cli/dg_cli_tests/cli_tests/test_list_commands.py b/python_modules/libraries/dg-cli/dg_cli_tests/cli_tests/test_list_commands.py new file mode 100644 index 0000000000000..e311c43455c39 --- /dev/null +++ b/python_modules/libraries/dg-cli/dg_cli_tests/cli_tests/test_list_commands.py @@ -0,0 +1,88 @@ +import sys +from pathlib import Path + +from click.testing import CliRunner +from dg_cli import __file__ as dg_cli_init_py +from dg_cli.cli.generate import generate_code_location_command, generate_component_command +from dg_cli.cli.list import ( + list_code_locations_command, + list_component_types_command, + list_components_command, +) + + +def ensure_dg_cli_tests_import() -> None: + dg_cli_package_root = (Path(dg_cli_init_py) / ".." / "..").resolve() + assert (dg_cli_package_root / "dg_cli_tests").exists(), "Could not find dg_cli where expected" + sys.path.append(dg_cli_package_root.as_posix()) + + +ensure_dg_cli_tests_import() + +from dg_cli_tests.cli_tests.test_generate_commands import ( + isolated_example_code_location_bar, + isolated_example_code_location_bar_with_component_type_baz, + isolated_example_deployment_foo, +) + + +def test_list_code_locations_success(): + runner = CliRunner() + with isolated_example_deployment_foo(runner): + runner.invoke(generate_code_location_command, ["foo"]) + runner.invoke(generate_code_location_command, ["bar"]) + result = runner.invoke(list_code_locations_command) + assert result.exit_code == 0 + assert result.output == "bar\nfoo\n" + + +def test_list_code_locations_outside_deployment_fails() -> None: + runner = CliRunner() + with runner.isolated_filesystem(): + result = runner.invoke(list_code_locations_command) + assert result.exit_code != 0 + assert "must be run inside a Dagster deployment project" in result.output + + +def test_list_component_types_success(): + runner = CliRunner() + with isolated_example_code_location_bar(runner): + result = runner.invoke(list_component_types_command) + assert result.exit_code == 0 + assert ( + result.output + == "\n".join( + [ + "dagster_components.dbt_project", + "dagster_components.pipes_subprocess_script_collection", + "dagster_components.sling_replication", + ] + ) + + "\n" + ) + + +def test_list_component_types_outside_code_location_fails() -> None: + runner = CliRunner() + with runner.isolated_filesystem(): + result = runner.invoke(list_component_types_command) + assert result.exit_code != 0 + assert "must be run inside a Dagster code location project" in result.output + + +def test_list_components_succeeds(): + runner = CliRunner() + # with isolated_example_code_location_bar(runner): + with isolated_example_code_location_bar_with_component_type_baz(runner): + result = runner.invoke(list_components_command) + runner.invoke(generate_component_command, ["bar.baz", "qux"]) + result = runner.invoke(list_components_command) + assert result.output == "qux\n" + + +def test_list_components_command_outside_code_location_fails() -> None: + runner = CliRunner() + with runner.isolated_filesystem(): + result = runner.invoke(list_components_command) + assert result.exit_code != 0 + assert "must be run inside a Dagster code location project" in result.output diff --git a/python_modules/libraries/dg-cli/pytest.ini b/python_modules/libraries/dg-cli/pytest.ini new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python_modules/libraries/dg-cli/setup.cfg b/python_modules/libraries/dg-cli/setup.cfg new file mode 100644 index 0000000000000..c267d25b3ad25 --- /dev/null +++ b/python_modules/libraries/dg-cli/setup.cfg @@ -0,0 +1,9 @@ +[metadata] +license_files = LICENSE + +[check-manifest] +ignore = + .coveragerc + tox.ini + pytest.ini + dg_cli_tests/** diff --git a/python_modules/libraries/dg-cli/setup.py b/python_modules/libraries/dg-cli/setup.py new file mode 100644 index 0000000000000..a662a1fd69cce --- /dev/null +++ b/python_modules/libraries/dg-cli/setup.py @@ -0,0 +1,51 @@ +from pathlib import Path +from typing import Dict + +from setuptools import find_packages, setup + + +def get_version() -> str: + version: Dict[str, str] = {} + with open(Path(__file__).parent / "dg_cli/version.py", encoding="utf8") as fp: + exec(fp.read(), version) + + return version["__version__"] + + +ver = get_version() +# dont pin dev installs to avoid pip dep resolver issues +pin = "" if ver == "1!0+dev" else f"=={ver}" +setup( + name="dagster-dg", + version=get_version(), + author="Dagster Labs", + author_email="hello@dagsterlabs.com", + license="Apache-2.0", + description="", # TODO - fill out description + url=("https://github.com/dagster-io/dagster/tree/master/python_modules/libraries/dg-cli"), + classifiers=[ + "Programming Language :: Python :: 3.9", + "Programming Language :: Python :: 3.10", + "Programming Language :: Python :: 3.11", + "Programming Language :: Python :: 3.12", + "License :: OSI Approved :: Apache Software License", + "Operating System :: OS Independent", + ], + packages=find_packages(exclude=["dg_cli_tests*"]), + install_requires=[ + "Jinja2", + "tomli", + "click>=8", + "typing_extensions>=4.4.0,<5", + ], + include_package_data=True, + zip_safe=False, + entry_points={ + "console_scripts": [ + "dg = dg_cli.cli:main", + ] + }, + extras_require={ + "test": ["pytest"], + }, +) diff --git a/python_modules/libraries/dg-cli/tox.ini b/python_modules/libraries/dg-cli/tox.ini new file mode 100644 index 0000000000000..90d009349e42b --- /dev/null +++ b/python_modules/libraries/dg-cli/tox.ini @@ -0,0 +1,19 @@ +[tox] +skipsdist = true + +[testenv] +download = True +passenv = + CI_* + COVERALLS_REPO_TOKEN + DAGSTER_GIT_REPO_DIR + BUILDKITE* +install_command = uv pip install {opts} {packages} +deps = + -e .[test] +allowlist_externals = + /bin/bash + uv +commands = + !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' + pytest ./dg_cli_tests -vv {posargs}
{isJob ? 'Job' : 'Pipeline'} nameTagsConfigurationTargetActions