Skip to content

Commit

Permalink
Update blocks to dispatch instance creation using slugs (#6622)
Browse files Browse the repository at this point in the history
* Update blocks to dispatch instance creation using slugs

* Update repr to only include data fields

* Fix document test

* Fix secret tests

* Add more test coverage for behavior after dispatch

* Add load test

* Add workaround and additional test coverage

* Exclude type slug from infrastructure submission mock
  • Loading branch information
zanieb committed Sep 7, 2022
1 parent c2a76aa commit 908fa00
Show file tree
Hide file tree
Showing 3 changed files with 228 additions and 8 deletions.
45 changes: 45 additions & 0 deletions src/prefect/blocks/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,13 @@ def __init__(self, *args, **kwargs):
def __str__(self) -> str:
return self.__repr__()

def __repr_args__(self):
repr_args = super().__repr_args__()
data_keys = self.schema()["properties"].keys()
return [
(key, value) for key, value in repr_args if key is None or key in data_keys
]

def block_initialization(self) -> None:
pass

Expand All @@ -185,6 +192,7 @@ def block_initialization(self) -> None:
# type name will default to the class name.
_block_type_name: Optional[str] = None
_block_type_slug: Optional[str] = None

# Attributes used to set properties on a block type when registered
# with Orion.
_logo_url: Optional[HttpUrl] = None
Expand Down Expand Up @@ -764,3 +772,40 @@ async def save(self, name: str, overwrite: bool = False):
document_id = await self._save(name=name, overwrite=overwrite)

return document_id

def _iter(self, *, include=None, exclude=None, **kwargs):
# Injects the `block_type_slug` into serialized payloads for dispatch
for key_value in super()._iter(include=include, exclude=exclude, **kwargs):
yield key_value

# Respect inclusion and exclusion still
if include and "block_type_slug" not in include:
return
if exclude and "block_type_slug" in exclude:
return

yield "block_type_slug", self.get_block_type_slug()

def __new__(cls: Type[Self], **kwargs) -> Self:
"""
Create an instance of the Block subclass type if a `block_type_slug` is
present in the data payload.
"""
block_type_slug = kwargs.pop("block_type_slug", None)
if block_type_slug:
subcls = lookup_type(cls, dispatch_key=block_type_slug)
m = super().__new__(subcls)
# NOTE: This is a workaround for an obscure issue where copied models were
# missing attributes. This pattern is from Pydantic's
# `BaseModel._copy_and_set_values`.
# The issue this fixes could not be reproduced in unit tests that
# directly targeted dispatch handling and was only observed when
# copying then saving infrastructure blocks on deployment models.
object.__setattr__(m, "__dict__", kwargs)
object.__setattr__(m, "__fields_set__", set(kwargs.keys()))
return m
else:
m = super().__new__(cls)
object.__setattr__(m, "__dict__", kwargs)
object.__setattr__(m, "__fields_set__", set(kwargs.keys()))
return m
189 changes: 182 additions & 7 deletions tests/blocks/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
from uuid import UUID, uuid4

import pytest
from pydantic import Field, SecretBytes, SecretStr
from pydantic import BaseModel, Field, SecretBytes, SecretStr

import prefect
from prefect.blocks.core import Block, InvalidBlockRegistration
Expand Down Expand Up @@ -181,10 +181,18 @@ class SecretBlock(Block):
assert isinstance(blockdoc.data["y"], SecretBytes)

json_blockdoc = json.loads(blockdoc.json())
assert json_blockdoc["data"] == {"x": "**********", "y": "**********", "z": "z"}
assert json_blockdoc["data"] == {
"x": "**********",
"y": "**********",
"z": "z",
}

json_blockdoc_with_secrets = json.loads(blockdoc.json(include_secrets=True))
assert json_blockdoc_with_secrets["data"] == {"x": "x", "y": "y", "z": "z"}
assert json_blockdoc_with_secrets["data"] == {
"x": "x",
"y": "y",
"z": "z",
}

def test_create_nested_api_block_with_secret_values_are_obfuscated_by_default(self):
class Child(Block):
Expand All @@ -209,14 +217,20 @@ class Parent(Block):
assert json_blockdoc["data"] == {
"a": "**********",
"b": "b",
"child": {"a": "**********", "b": "b"},
# The child includes the type slug because it is not a block document
"child": {
"a": "**********",
"b": "b",
"block_type_slug": "child",
},
}

json_blockdoc_with_secrets = json.loads(blockdoc.json(include_secrets=True))
assert json_blockdoc_with_secrets["data"] == {
"a": "a",
"b": "b",
"child": {"a": "a", "b": "b"},
# The child includes the type slug because it is not a block document
"child": {"a": "a", "b": "b", "block_type_slug": "child"},
}

def test_registering_blocks_with_capabilities(self):
Expand Down Expand Up @@ -707,18 +721,21 @@ class E(Block):
"_block_document_id": middle_block_document_1.id,
"_block_document_name": "middle-block-document-1",
"_is_anonymous": False,
"block_type_slug": "c",
}
assert block_instance.d.dict() == {
"b": {
"x": 1,
"_block_document_id": inner_block_document.id,
"_block_document_name": "inner-block-document",
"_is_anonymous": False,
"block_type_slug": "b",
},
"z": "ztop",
"_block_document_id": middle_block_document_2.id,
"_block_document_name": "middle-block-document-2",
"_is_anonymous": False,
"block_type_slug": "d",
}

async def test_create_block_from_nonexistent_name(self, test_block):
Expand Down Expand Up @@ -1152,7 +1169,7 @@ class Parent(Block):
assert db_block_without_secrets.data == {
"a": obfuscate_string("a"),
"b": "b",
"child": {"a": obfuscate_string("a"), "b": "b"},
"child": {"a": obfuscate_string("a"), "b": "b", "block_type_slug": "child"},
}

# read from DB with secrets
Expand All @@ -1161,7 +1178,11 @@ class Parent(Block):
block_document_id=block._block_document_id,
include_secrets=True,
)
assert db_block.data == {"a": "a", "b": "b", "child": {"a": "a", "b": "b"}}
assert db_block.data == {
"a": "a",
"b": "b",
"child": {"a": "a", "b": "b", "block_type_slug": "child"},
}

# load block with secrets
api_block = await Parent.load("secret-block")
Expand Down Expand Up @@ -1691,3 +1712,157 @@ async def my_flow():

result = await my_flow()
assert result == 1000000


# Define types for `TestTypeDispatch`


class BaseBlock(Block):
base: int = 0


class ParentModel(BaseModel):
block: BaseBlock


class AChildBlock(BaseBlock):
a: int = 1


class BChildBlock(BaseBlock):
b: int = 2


class TestTypeDispatch:
def test_block_type_slug_is_included_in_dict(self):
assert "block_type_slug" in AChildBlock().dict()

def test_block_type_slug_respects_exclude(self):
assert "block_type_slug" not in AChildBlock().dict(exclude={"block_type_slug"})

def test_block_type_slug_respects_include(self):
assert "block_type_slug" not in AChildBlock().dict(include={"a"})

async def test_block_type_slug_excluded_from_document(self, orion_client):
await AChildBlock.register_type_and_schema(client=orion_client)
document = AChildBlock()._to_block_document(name="foo")
assert "block_type_slug" not in document.data

def test_base_parse_works_for_base_instance(self):
block = BaseBlock.parse_obj(BaseBlock().dict())
assert type(block) == BaseBlock

block = BaseBlock.parse_obj(BaseBlock().dict())
assert type(block) == BaseBlock

def test_base_parse_creates_child_instance_from_dict(self):
block = BaseBlock.parse_obj(AChildBlock().dict())
assert type(block) == AChildBlock

block = BaseBlock.parse_obj(BChildBlock().dict())
assert type(block) == BChildBlock

def test_base_parse_creates_child_instance_from_json(self):
block = BaseBlock.parse_raw(AChildBlock().json())
assert type(block) == AChildBlock

block = BaseBlock.parse_raw(BChildBlock().json())
assert type(block) == BChildBlock

def test_base_parse_retains_default_attributes(self):
block = BaseBlock.parse_obj(AChildBlock().dict())
assert block.base == 0
assert block.a == 1

def test_base_parse_retains_set_child_attributes(self):
block = BaseBlock.parse_obj(BChildBlock(b=3).dict())
assert block.base == 0
assert block.b == 3

def test_base_parse_retains_set_base_attributes(self):
block = BaseBlock.parse_obj(BChildBlock(base=1).dict())
assert block.base == 1
assert block.b == 2

def test_base_field_creates_child_instance_from_object(self):
model = ParentModel(block=AChildBlock())
assert type(model.block) == AChildBlock

model = ParentModel(block=BChildBlock())
assert type(model.block) == BChildBlock

def test_base_field_creates_child_instance_from_dict(self):
model = ParentModel(block=AChildBlock().dict())
assert type(model.block) == AChildBlock

model = ParentModel(block=BChildBlock().dict())
assert type(model.block) == BChildBlock

def test_created_block_has_pydantic_attributes(self):
block = BaseBlock.parse_obj(AChildBlock().dict())
assert block.__fields_set__

def test_created_block_can_be_copied(self):
block = BaseBlock.parse_obj(AChildBlock().dict())
block_copy = block.copy()
assert block == block_copy

async def test_created_block_can_be_saved(self):
block = BaseBlock.parse_obj(AChildBlock().dict())
assert await block.save("test")

async def test_created_block_can_be_saved_then_loaded(self):
block = BaseBlock.parse_obj(AChildBlock().dict())
await block.save("test")
new_block = await block.load("test")
assert block == new_block
assert new_block.__fields_set__

def test_created_block_fields_set(self):
expected = {"base", "block_type_slug", "a"}

block = BaseBlock.parse_obj(AChildBlock().dict())
assert block.__fields_set__ == expected
assert block.a == 1

block = BaseBlock.parse_obj(AChildBlock(a=2).dict())
assert block.__fields_set__ == expected
assert block.a == 2

block = block.copy()
assert block.__fields_set__ == expected
assert block.a == 2

def test_base_field_creates_child_instance_with_union(self):
class UnionParentModel(BaseModel):
block: Union[AChildBlock, BChildBlock]

model = UnionParentModel(block=AChildBlock(a=3).dict())
assert type(model.block) == AChildBlock

# Assignment with a copy works still
model.block = model.block.copy()
assert type(model.block) == AChildBlock
assert model.block

model = UnionParentModel(block=BChildBlock(b=4).dict())
assert type(model.block) == BChildBlock

def test_base_field_creates_child_instance_with_assignment_validation(self):
class AssignmentParentModel(BaseModel):
block: BaseBlock

class Config:
validate_assignment = True

model = AssignmentParentModel(block=AChildBlock(a=3).dict())
assert type(model.block) == AChildBlock
assert model.block.a == 3

model.block = model.block.copy()
assert type(model.block) == AChildBlock
assert model.block.a == 3

model.block = BChildBlock(b=4).dict()
assert type(model.block) == BChildBlock
assert model.block.b == 4
2 changes: 1 addition & 1 deletion tests/infrastructure/test_submission.py
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ class MockInfrastructure(Infrastructure):
async def run(self, task_status=None):
if task_status:
task_status.started()
self._run(self.dict())
self._run(self.dict(exclude={"block_type_slug"}))

def preview(self):
return self.json()
Expand Down

0 comments on commit 908fa00

Please sign in to comment.