Skip to content

Commit ec58c3b

Browse files
authored
[dg] Serialize component type info with code location, add GQL to power in-app docs (#29167)
## Summary Packages up component type JSON into the repository metadata of `load_defs`, which can be accessed via GraphQL through the new `locationDocsJson` query, which returns contents identical to the generated JSON which powers `dg docs serve`.
1 parent 9d5623b commit ec58c3b

File tree

18 files changed

+454
-184
lines changed

18 files changed

+454
-184
lines changed

Diff for: js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json

+1-1
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

Diff for: js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql

+9
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

Diff for: js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts

+31
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

Diff for: python_modules/dagster-graphql/dagster_graphql/schema/env_vars.py

+13
Original file line numberDiff line numberDiff line change
@@ -58,3 +58,16 @@ class GrapheneEnvVarWithConsumersListOrError(graphene.Union):
5858
class Meta:
5959
types = (GrapheneEnvVarWithConsumersList, GraphenePythonError)
6060
name = "EnvVarWithConsumersOrError"
61+
62+
63+
class GrapheneLocationDocsJson(graphene.ObjectType):
64+
class Meta:
65+
name = "LocationDocsJson"
66+
67+
json = graphene.NonNull(graphene.JSONString)
68+
69+
70+
class GrapheneLocationDocsJsonOrError(graphene.Union):
71+
class Meta:
72+
types = (GrapheneLocationDocsJson, GraphenePythonError)
73+
name = "LocationDocsJsonOrError"

Diff for: python_modules/dagster-graphql/dagster_graphql/schema/roots/query.py

+33-1
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
from dagster._core.scheduler.instigation import InstigatorStatus, InstigatorType
2222
from dagster._core.storage.event_log.base import AssetRecord
2323
from dagster._core.workspace.permissions import Permissions
24+
from dagster.components.core.load_defs import PLUGIN_COMPONENT_TYPES_JSON_METADATA_KEY
2425

2526
from dagster_graphql.implementation.execution.backfill import get_asset_backfill_preview
2627
from dagster_graphql.implementation.external import (
@@ -118,7 +119,11 @@
118119
GraphenePartitionBackfillsOrError,
119120
)
120121
from dagster_graphql.schema.entity_key import GrapheneAssetKey
121-
from dagster_graphql.schema.env_vars import GrapheneEnvVarWithConsumersListOrError
122+
from dagster_graphql.schema.env_vars import (
123+
GrapheneEnvVarWithConsumersListOrError,
124+
GrapheneLocationDocsJson,
125+
GrapheneLocationDocsJsonOrError,
126+
)
122127
from dagster_graphql.schema.external import (
123128
GrapheneRepositoriesOrError,
124129
GrapheneRepositoryConnection,
@@ -312,6 +317,12 @@ class Meta:
312317
description="Retrieve all the utilized environment variables for the given repo.",
313318
)
314319

320+
locationDocsJsonOrError = graphene.Field(
321+
graphene.NonNull(GrapheneLocationDocsJsonOrError),
322+
repositorySelector=graphene.NonNull(GrapheneRepositorySelector),
323+
description="Retrieves JSON blob to drive integrated code location docs.",
324+
)
325+
315326
sensorOrError = graphene.Field(
316327
graphene.NonNull(GrapheneSensorOrError),
317328
sensorSelector=graphene.NonNull(GrapheneSensorSelector),
@@ -767,6 +778,27 @@ def resolve_utilizedEnvVarsOrError(self, graphene_info: ResolveInfo, **kwargs):
767778
RepositorySelector.from_graphql_input(kwargs.get("repositorySelector")),
768779
)
769780

781+
@capture_error
782+
def resolve_locationDocsJsonOrError(
783+
self, graphene_info: ResolveInfo, repositorySelector: GrapheneRepositorySelector
784+
) -> GrapheneLocationDocsJson:
785+
repo_selector = RepositorySelector.from_graphql_input(repositorySelector)
786+
787+
location = graphene_info.context.get_code_location(repo_selector.location_name)
788+
repository = location.get_repository(repo_selector.repository_name)
789+
plugin_docs_json = (
790+
cast(
791+
list,
792+
repository.repository_snap.metadata.get(
793+
PLUGIN_COMPONENT_TYPES_JSON_METADATA_KEY, [[]]
794+
),
795+
)[0]
796+
if repository.repository_snap.metadata
797+
else []
798+
)
799+
800+
return GrapheneLocationDocsJson(json=plugin_docs_json)
801+
770802
@capture_error
771803
def resolve_sensorOrError(
772804
self, graphene_info: ResolveInfo, sensorSelector: GrapheneRepositorySelector

Diff for: python_modules/dagster-graphql/dagster_graphql_tests/graphql/components/__init__.py

Whitespace-only changes.

Diff for: python_modules/dagster-graphql/dagster_graphql_tests/graphql/components/defs/__init__.py

Whitespace-only changes.
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,82 @@
1+
import json
2+
from unittest import mock
3+
4+
from dagster import Definitions
5+
from dagster._core.definitions.repository_definition.repository_definition import (
6+
RepositoryDefinition,
7+
)
8+
from dagster._core.test_utils import ensure_dagster_tests_import, instance_for_test
9+
from dagster_graphql.test.utils import define_out_of_process_context, execute_dagster_graphql
10+
from dagster_shared.serdes.objects.package_entry import PluginObjectKey
11+
12+
ensure_dagster_tests_import()
13+
14+
GET_DOCS_JSON_QUERY = """
15+
query GetDocsJson {
16+
locationDocsJsonOrError(repositorySelector: {repositoryLocationName: "test_location", repositoryName: "__repository__"}) {
17+
__typename
18+
... on LocationDocsJson {
19+
json
20+
}
21+
... on PythonError {
22+
message
23+
}
24+
}
25+
}
26+
"""
27+
28+
29+
def get_empty_repo() -> RepositoryDefinition:
30+
return Definitions().get_repository_def()
31+
32+
33+
def test_get_empty_docs_json():
34+
with (
35+
instance_for_test() as instance,
36+
define_out_of_process_context(__file__, "get_empty_repo", instance) as context,
37+
):
38+
get_docs_json_result = execute_dagster_graphql(context, GET_DOCS_JSON_QUERY)
39+
assert (
40+
get_docs_json_result.data["locationDocsJsonOrError"]["__typename"] == "LocationDocsJson"
41+
)
42+
assert get_docs_json_result.data["locationDocsJsonOrError"]["json"] is not None
43+
44+
json_contents = json.loads(get_docs_json_result.data["locationDocsJsonOrError"]["json"])
45+
assert len(json_contents) == 0
46+
47+
48+
def get_components_repo() -> RepositoryDefinition:
49+
with mock.patch(
50+
"dagster.components.core.package_entry.discover_entry_point_package_objects"
51+
) as mock_discover_entry_point_package_objects:
52+
import dagster_test.components
53+
from dagster.components.core.package_entry import get_package_objects_in_module
54+
55+
from dagster_graphql_tests.graphql.components import defs as defs
56+
57+
objects = {}
58+
for name, obj in get_package_objects_in_module(dagster_test.components):
59+
key = PluginObjectKey(name=name, namespace="dagster_test")
60+
objects[key] = obj
61+
62+
mock_discover_entry_point_package_objects.return_value = objects
63+
64+
from dagster.components.core.load_defs import load_defs
65+
66+
return load_defs(defs).get_repository_def()
67+
68+
69+
def test_get_docs_json():
70+
with (
71+
instance_for_test() as instance,
72+
define_out_of_process_context(__file__, "get_components_repo", instance) as context,
73+
):
74+
get_docs_json_result = execute_dagster_graphql(context, GET_DOCS_JSON_QUERY)
75+
assert (
76+
get_docs_json_result.data["locationDocsJsonOrError"]["__typename"] == "LocationDocsJson"
77+
)
78+
assert get_docs_json_result.data["locationDocsJsonOrError"]["json"] is not None
79+
80+
json_contents = json.loads(get_docs_json_result.data["locationDocsJsonOrError"]["json"])
81+
assert len(json_contents) == 1
82+
assert json_contents[0]["name"] == "dagster_test"

Diff for: python_modules/dagster-graphql/setup.py

+1
Original file line numberDiff line numberDiff line change
@@ -40,5 +40,6 @@ def get_version() -> str:
4040
"requests",
4141
"starlette", # used for run_in_threadpool utility fn
4242
],
43+
extras_require={"test": ["dagster-test"]},
4344
entry_points={"console_scripts": ["dagster-graphql = dagster_graphql.cli:main"]},
4445
)

Diff for: python_modules/dagster-graphql/tox.ini

+2-1
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,8 @@ deps =
1717
-e ../dagster-pipes
1818
-e ../libraries/dagster-shared
1919
postgres: -e ../libraries/dagster-postgres
20-
-e .
20+
-e ../dagster-test
21+
-e .[test]
2122
allowlist_externals =
2223
/bin/bash
2324
uv

Diff for: python_modules/dagster/dagster/components/core/load_defs.py

+14-1
Original file line numberDiff line numberDiff line change
@@ -2,12 +2,16 @@
22
from pathlib import Path
33
from types import ModuleType
44

5+
from dagster_shared.serdes.objects.package_entry import json_for_all_components
6+
57
from dagster._annotations import deprecated
68
from dagster._core.definitions.definitions_class import Definitions
79
from dagster._core.errors import DagsterInvalidDefinitionError
810
from dagster._utils.warnings import suppress_dagster_warnings
911
from dagster.components.core.context import ComponentLoadContext, use_component_load_context
1012

13+
PLUGIN_COMPONENT_TYPES_JSON_METADATA_KEY = "plugin_component_types_json"
14+
1115

1216
@deprecated(breaking_version="0.2.0")
1317
@suppress_dagster_warnings
@@ -36,12 +40,21 @@ def load_defs(defs_root: ModuleType) -> Definitions:
3640
to apply to the definitions.
3741
"""
3842
from dagster.components.core.defs_module import DefsModuleComponent
43+
from dagster.components.core.package_entry import discover_entry_point_package_objects
44+
from dagster.components.core.snapshot import get_package_entry_snap
3945

4046
# create a top-level DefsModule component from the root module
4147
context = ComponentLoadContext.for_module(defs_root)
4248
root_component = DefsModuleComponent.from_context(context)
4349
if root_component is None:
4450
raise DagsterInvalidDefinitionError("Could not resolve root module to a component.")
4551

52+
library_objects = discover_entry_point_package_objects()
53+
snaps = [get_package_entry_snap(key, obj) for key, obj in library_objects.items()]
54+
components_json = json_for_all_components(snaps)
55+
4656
with use_component_load_context(context):
47-
return root_component.build_defs(context)
57+
return Definitions.merge(
58+
root_component.build_defs(context),
59+
Definitions(metadata={PLUGIN_COMPONENT_TYPES_JSON_METADATA_KEY: components_json}),
60+
)

Diff for: python_modules/libraries/dagster-dg/dagster_dg/cli/docs.py

+11-4
Original file line numberDiff line numberDiff line change
@@ -8,13 +8,13 @@
88
from typing import Optional
99

1010
import click
11+
from dagster_shared.serdes.objects.package_entry import json_for_all_components
1112
from yaspin import yaspin
1213

1314
from dagster_dg.cli.shared_options import dg_global_options
1415
from dagster_dg.component import PluginObjectKey, RemotePluginRegistry
1516
from dagster_dg.config import normalize_cli_config
1617
from dagster_dg.context import DgContext
17-
from dagster_dg.docs import json_for_all_components
1818
from dagster_dg.utils import DgClickCommand, DgClickGroup, exit_with_error, pushd
1919
from dagster_dg.utils.telemetry import cli_telemetry_wrapper
2020

@@ -27,7 +27,9 @@
2727
/ "dg-docs-site"
2828
)
2929
DOCS_DIR = Path(__file__).parent.parent / "docs" / "packages" / "dg-docs-site"
30-
ACTIVE_DOCS_DIR = DOCS_DIR if DOCS_DIR.exists() else DEV_DOCS_DIR
30+
ACTIVE_DOCS_DIR = (
31+
DOCS_DIR if DOCS_DIR.exists() and (DOCS_DIR / "package.json").exists() else DEV_DOCS_DIR
32+
)
3133

3234
DOCS_JSON_PATH = ACTIVE_DOCS_DIR / "contents" / "generated.json"
3335

@@ -65,7 +67,9 @@ def serve_docs_command(
6567

6668
with pushd(ACTIVE_DOCS_DIR):
6769
DOCS_JSON_PATH.parent.mkdir(parents=True, exist_ok=True)
68-
DOCS_JSON_PATH.write_text(json.dumps(json_for_all_components(registry), indent=2))
70+
DOCS_JSON_PATH.write_text(
71+
json.dumps(json_for_all_components([v for _, v in registry.items()]), indent=2)
72+
)
6973
with yaspin(text="Verifying docs dependencies", color="blue") as spinner:
7074
yes = subprocess.Popen(["yes", "y"], stdout=subprocess.PIPE)
7175
try:
@@ -122,7 +126,9 @@ def build_docs_command(
122126

123127
with pushd(ACTIVE_DOCS_DIR):
124128
DOCS_JSON_PATH.parent.mkdir(parents=True, exist_ok=True)
125-
DOCS_JSON_PATH.write_text(json.dumps(json_for_all_components(registry), indent=2))
129+
DOCS_JSON_PATH.write_text(
130+
json.dumps(json_for_all_components([v for _, v in registry.items()]), indent=2)
131+
)
126132
with yaspin(text="Verifying docs dependencies", color="blue") as spinner:
127133
yes = subprocess.Popen(["yes", "y"], stdout=subprocess.PIPE)
128134
try:
@@ -132,6 +138,7 @@ def build_docs_command(
132138
spinner.ok("✓")
133139

134140
spinner = yaspin(text="Building docs", color="blue")
141+
spinner.start()
135142
subprocess.check_output(["yarn", "build"])
136143
spinner.ok("✓")
137144

0 commit comments

Comments
 (0)