Skip to content

Commit

Permalink
serializable RemoteAssetNodes (#24827)
Browse files Browse the repository at this point in the history
Make the global and repository scoped `RemoteAssetNode` serializable so
we can persist/cache the resolved global asset node of a workspace.

## How I Tested These Changes

added test
  • Loading branch information
alangenfeld authored Oct 15, 2024
1 parent d7c8769 commit 0df5139
Show file tree
Hide file tree
Showing 4 changed files with 28 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
from dagster._core.remote_representation.handle import InstigatorHandle, RepositoryHandle
from dagster._core.workspace.workspace import WorkspaceSnapshot
from dagster._record import ImportFrom, record
from dagster._serdes.serdes import whitelist_for_serdes
from dagster._utils.cached_method import cached_method

if TYPE_CHECKING:
Expand Down Expand Up @@ -124,6 +125,7 @@ def job_names(self) -> Sequence[str]:
)


@whitelist_for_serdes
@record
class RemoteRepositoryAssetNode(RemoteAssetNode):
"""Asset nodes from a single RemoteRepository."""
Expand Down Expand Up @@ -191,6 +193,7 @@ def auto_observe_interval_minutes(self) -> Optional[float]:
return self.asset_node_snap.auto_observe_interval_minutes


@whitelist_for_serdes
@record
class RepositoryScopedAssetInfo:
"""RemoteRepositoryAssetNode paired with additional information from that repository.
Expand All @@ -207,6 +210,7 @@ def handle(self) -> RepositoryHandle:
return self.asset_node.repository_handle


@whitelist_for_serdes
@record
class RemoteWorkspaceAssetNode(RemoteAssetNode):
"""Asset nodes constructed from a WorkspaceSnapshot, containing nodes from potentially several RemoteRepositories."""
Expand Down Expand Up @@ -315,7 +319,6 @@ def resolve_to_singular_repo_scoped_node(self) -> "RemoteRepositoryAssetNode":
# materialization node was previously preferred over the observable node. This is a
# temporary measure until we can appropriately scope the accessors that could apply to
# either a materialization or observation node.

# This property supports existing behavior but it should be phased out, because it relies on
# materialization nodes shadowing observation nodes that would otherwise be exposed.
return next(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,11 +12,13 @@
RemoteRepositoryOrigin,
)
from dagster._record import IHaveNew, record, record_custom
from dagster._serdes.serdes import whitelist_for_serdes

if TYPE_CHECKING:
from dagster._core.remote_representation.code_location import CodeLocation


@whitelist_for_serdes
@record
class RepositoryHandle:
repository_name: str
Expand Down
8 changes: 8 additions & 0 deletions python_modules/dagster/dagster/_serdes/serdes.py
Original file line number Diff line number Diff line change
Expand Up @@ -871,6 +871,14 @@ def pack_value(
) -> Sequence[JsonSerializableValue]: ...


@overload
def pack_value(
val: PackableValue,
whitelist_map: WhitelistMap = ...,
descent_path: Optional[str] = ...,
) -> JsonSerializableValue: ...


def pack_value(
val: PackableValue,
whitelist_map: WhitelistMap = _WHITELIST_MAP,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
from dagster._core.remote_representation.external_data import RepositorySnap
from dagster._core.remote_representation.handle import RepositoryHandle
from dagster._core.test_utils import freeze_time, instance_for_test, mock_workspace_from_repos
from dagster._serdes.serdes import deserialize_value, serialize_value
from dagster._time import create_datetime, get_current_datetime


Expand Down Expand Up @@ -894,3 +895,16 @@ def repo_b():
asset_graph.get_partition_mapping(key=b.key, parent_asset_key=a.key),
TimeWindowPartitionMapping,
)


def test_serdes() -> None:
@asset
def a(): ...

@repository
def repo():
return [a]

asset_graph = mock_workspace_from_repos([repo]).asset_graph
for node in asset_graph.asset_nodes:
assert node == deserialize_value(serialize_value(node))

0 comments on commit 0df5139

Please sign in to comment.