Skip to content

Commit

Permalink
switch arg name for metadata value class (#6675)
Browse files Browse the repository at this point in the history
  • Loading branch information
smackesey committed Mar 7, 2022
1 parent 2910338 commit ed210ae
Show file tree
Hide file tree
Showing 2 changed files with 88 additions and 32 deletions.
109 changes: 78 additions & 31 deletions python_modules/dagster/dagster/core/definitions/metadata/__init__.py
Original file line number Diff line number Diff line change
@@ -1,11 +1,17 @@
import functools
import os
import re
from typing import TYPE_CHECKING, Any, Callable, Dict, List, NamedTuple, Optional, Union, cast

from dagster import check, seven
from dagster.core.errors import DagsterInvalidMetadata
from dagster.serdes import whitelist_for_serdes
from dagster.utils.backcompat import deprecation_warning, experimental, experimental_class_warning
from dagster.utils.backcompat import (
canonicalize_backcompat_args,
deprecation_warning,
experimental,
experimental_class_warning,
)

from .table import TableColumn, TableColumnConstraints, TableConstraints, TableRecord, TableSchema

Expand Down Expand Up @@ -80,45 +86,50 @@ def normalize_metadata(
]


def package_metadata_value(label: str, value: RawMetadataValue) -> "MetadataEntry":
check.str_param(label, "label")

if isinstance(value, (MetadataEntry, PartitionMetadataEntry)):
raise DagsterInvalidMetadata(
f"Expected a metadata value, found an instance of {value.__class__.__name__}. Consider "
"instead using a MetadataValue wrapper for the value."
)

if isinstance(value, MetadataValue):
return MetadataEntry(label, None, value)
def normalize_metadata_value(raw_value: RawMetadataValue):

if isinstance(value, str):
return MetadataEntry.text(value, label)

if isinstance(value, float):
return MetadataEntry.float(value, label)

if isinstance(value, int):
return MetadataEntry.int(value, label)

if isinstance(value, dict):
if isinstance(raw_value, MetadataValue):
return raw_value
elif isinstance(raw_value, str):
return MetadataValue.text(raw_value)
elif isinstance(raw_value, float):
return MetadataValue.float(raw_value)
elif isinstance(raw_value, int):
return MetadataValue.int(raw_value)
elif isinstance(raw_value, dict):
try:
# check that the value is JSON serializable
seven.dumps(value)
return MetadataEntry.json(value, label)
seven.dumps(raw_value)
return MetadataValue.json(raw_value)
except TypeError:
raise DagsterInvalidMetadata(
f'Could not resolve the metadata value for "{label}" to a JSON serializable value. '
"Value is a dictionary but is not JSON serializable. "
"Consider wrapping the value with the appropriate MetadataValue type."
)

raise DagsterInvalidMetadata(
f'Could not resolve the metadata value for "{label}" to a known type. '
f"Its type was {type(value)}. Consider wrapping the value with the appropriate "
f"Its type was {type(raw_value)}. Consider wrapping the value with the appropriate "
"MetadataValue type."
)


def package_metadata_value(label: str, raw_value: RawMetadataValue) -> "MetadataEntry":
check.str_param(label, "label")

if isinstance(raw_value, (MetadataEntry, PartitionMetadataEntry)):
raise DagsterInvalidMetadata(
f"Expected a metadata value, found an instance of {raw_value.__class__.__name__}. Consider "
"instead using a MetadataValue wrapper for the value."
)
try:
value = normalize_metadata_value(raw_value)
except DagsterInvalidMetadata as e:
raise DagsterInvalidMetadata(
f'Could not resolve the metadata value for "{label}" to a known type. {e}'
) from None
return MetadataEntry(label=label, value=value)


# ########################
# ##### METADATA VALUE
# ########################
Expand Down Expand Up @@ -719,7 +730,16 @@ def wrapper(*args, **kwargs):
deprecation_warning(
f"Function `MetadataEntry.{fn.__name__}`",
"0.15.0",
additional_warn_txt="In the future, construct `MetadataEntry` by calling the constructor directly and passing a `MetadataValue`.",
additional_warn_txt=re.sub(
r"\n\s*",
" ",
"""
The recommended way to supply metadata is to pass a `Dict[str,
MetadataValue]` to the `metadata` keyword argument. To construct `MetadataEntry`
directly, call constructor and pass a `MetadataValue`: `MetadataEntry(label="foo",
value=MetadataValue.text("bar")",
""",
),
)
return fn(*args, **kwargs)

Expand All @@ -728,6 +748,10 @@ def wrapper(*args, **kwargs):

# NOTE: This would better be implemented as a generic with `MetadataValue` set as a
# typevar, but as of 2022-01-25 mypy does not support generics on NamedTuple.
#
# NOTE: This currently stores value in the `entry_data` NamedTuple attribute. In the next release,
# we will change the name of the NamedTuple property to `value`, and need to implement custom
# serialization so that it continues to be saved as `entry_data` for backcompat purposes.
@whitelist_for_serdes(storage_name="EventMetadataEntry")
class MetadataEntry(
NamedTuple(
Expand All @@ -751,23 +775,46 @@ class MetadataEntry(
Args:
label (str): Short display label for this metadata entry.
description (Optional[str]): A human-readable description of this metadata entry.
entry_data (MetadataValue): Typed metadata entry data. The different types allow
value (MetadataValue): Typed metadata entry data. The different types allow
for customized display in tools like dagit.
"""

def __new__(cls, label: str, description: Optional[str], entry_data: "MetadataValue"):
def __new__(
cls,
label: str,
description: Optional[str] = None,
entry_data: Optional["RawMetadataValue"] = None,
value: Optional["RawMetadataValue"] = None,
):
if description is not None:
deprecation_warning(
'The "description" attribute on "MetadataEntry"',
"0.15.0",
)
value = cast(
RawMetadataValue,
canonicalize_backcompat_args(
new_val=value,
new_arg="value",
old_val=entry_data,
old_arg="entry_data",
breaking_version="0.15.0",
),
)
value = normalize_metadata_value(value)

return super(MetadataEntry, cls).__new__(
cls,
check.str_param(label, "label"),
check.opt_str_param(description, "description"),
check.inst_param(entry_data, "entry_data", MetadataValue),
check.inst_param(value, "value", MetadataValue),
)

@property
def value(self):
"""Alias of `entry_data`."""
return self.entry_data

@staticmethod
@deprecated_metadata_entry_constructor
def text(text: Optional[str], label: str, description: Optional[str] = None) -> "MetadataEntry":
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,14 @@ def solid_events_for_type(result, solid_name, event_type):
]


def test_metadata_entry_construction():
entry_1 = MetadataEntry("foo", value=MetadataValue.text("bar"))
entry_2 = MetadataEntry("foo", entry_data=MetadataValue.text("bar"))
assert entry_1.value == MetadataValue.text("bar")
assert entry_2.value == MetadataValue.text("bar")
assert entry_1 == entry_2


def test_metadata_asset_materialization():
@solid(output_defs=[])
def the_solid(_context):
Expand Down Expand Up @@ -166,7 +174,8 @@ def the_pipeline():
execute_pipeline(the_pipeline)

assert str(exc_info.value) == (
'Could not resolve the metadata value for "bad" to a JSON serializable value. '
'Could not resolve the metadata value for "bad" to a known type. '
"Value is a dictionary but is not JSON serializable. "
"Consider wrapping the value with the appropriate MetadataValue type."
)

Expand Down

0 comments on commit ed210ae

Please sign in to comment.