Skip to content

Commit

Permalink
Allow PathMetadataValue to accept os.PathLike input (#6920)
Browse files Browse the repository at this point in the history
* Allow PathMetadataValue to accept os.PathLike input (closes #6916)
  • Loading branch information
abkfenris committed Mar 9, 2022
1 parent e0dc13b commit 02679a1
Show file tree
Hide file tree
Showing 4 changed files with 85 additions and 12 deletions.
35 changes: 35 additions & 0 deletions python_modules/dagster/dagster/check/__init__.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
import inspect
from os import PathLike, fspath
from typing import (
AbstractSet,
Any,
Expand Down Expand Up @@ -1050,6 +1051,40 @@ def _check_tuple_items(
return obj_tuple


# ########################
# ##### PATH
# ########################


def path_param(obj: object, param_name: str) -> str:
if not isinstance(obj, (str, PathLike)):
raise _param_type_mismatch_exception(obj, (str, PathLike), param_name)
return fspath(obj)


@overload
def opt_path_param(obj: object, param_name: str, default: Union[str, PathLike]) -> str:
...


@overload
def opt_path_param(obj: object, param_name: str) -> Optional[str]:
...


def opt_path_param(
obj: object, param_name: str, default: Optional[Union[str, PathLike]] = None
) -> Optional[str]:
if obj is not None and not isinstance(obj, (str, PathLike)):
raise _param_type_mismatch_exception(obj, (str, PathLike), param_name)
if obj is not None:
return fspath(obj)

if obj is None and default is None:
return default
return fspath(default)


# ###################################################################################################
# ##### OTHER CHECKS
# ###################################################################################################
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,9 @@ def normalize_metadata_value(raw_value: RawMetadataValue):
"Consider wrapping the value with the appropriate MetadataValue type."
)

if isinstance(raw_value, os.PathLike):
return MetadataValue.path(raw_value)

raise DagsterInvalidMetadata(
f"Its type was {type(raw_value)}. Consider wrapping the value with the appropriate "
"MetadataValue type."
Expand Down Expand Up @@ -199,7 +202,7 @@ def emit_metadata(context):
return UrlMetadataValue(url)

@staticmethod
def path(path: str) -> "PathMetadataValue":
def path(path: Union[str, os.PathLike]) -> "PathMetadataValue":
"""Static constructor for a metadata value wrapping a path as
:py:class:`PathMetadataValue`. For example:
Expand Down Expand Up @@ -496,23 +499,17 @@ def __new__(cls, url: Optional[str]):

@whitelist_for_serdes(storage_name="PathMetadataEntryData")
class PathMetadataValue( # type: ignore
NamedTuple(
"_PathMetadataValue",
[
("path", Optional[str]),
],
),
MetadataValue,
NamedTuple("_PathMetadataValue", [("path", Optional[str])]), MetadataValue
):
"""Container class for path metadata entry data.
Args:
path (Optional[str]): The path as a string.
path (Optional[str]): The path as a string or conforming to os.PathLike.
"""

def __new__(cls, path: Optional[str]):
def __new__(cls, path: Optional[Union[str, os.PathLike]]):
return super(PathMetadataValue, cls).__new__(
cls, check.opt_str_param(path, "path", default="")
cls, check.opt_path_param(path, "path", default="")
)


Expand Down
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
from pathlib import Path

import pytest

from dagster import (
Expand All @@ -7,6 +9,7 @@
FloatMetadataValue,
IntMetadataValue,
MetadataValue,
PathMetadataValue,
PythonArtifactMetadataValue,
TextMetadataValue,
UrlMetadataValue,
Expand Down Expand Up @@ -57,6 +60,7 @@ def the_solid(_context):
"int": 22,
"url": MetadataValue.url("http://fake.com"),
"float": 0.1,
"path": MetadataValue.path(Path("/a/b.csv")),
"python": MetadataValue.python_artifact(MetadataValue),
},
)
Expand All @@ -75,14 +79,15 @@ def the_pipeline():
)
assert len(materialization_events) == 1
materialization = materialization_events[0].event_specific_data.materialization
assert len(materialization.metadata_entries) == 5
assert len(materialization.metadata_entries) == 6
entry_map = {
entry.label: entry.entry_data.__class__ for entry in materialization.metadata_entries
}
assert entry_map["text"] == TextMetadataValue
assert entry_map["int"] == IntMetadataValue
assert entry_map["url"] == UrlMetadataValue
assert entry_map["float"] == FloatMetadataValue
assert entry_map["path"] == PathMetadataValue
assert entry_map["python"] == PythonArtifactMetadataValue


Expand Down Expand Up @@ -158,6 +163,16 @@ def test_parse_invalid_metadata():
assert entries[0].entry_data == TextMetadataValue("[object] (unserializable)")


def test_parse_path_metadata():

metadata = {"path": Path("/a/b.csv")}

entries = normalize_metadata(metadata, [])
assert len(entries) == 1
assert entries[0].label == "path"
assert entries[0].entry_data == PathMetadataValue("/a/b.csv")


def test_bad_json_metadata_value():
@solid(output_defs=[])
def the_solid(context):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,32 @@ def test_opt_nonempty_str_param():
check.opt_nonempty_str_param(1, "str_param")


def test_path_param():
from pathlib import Path

assert check.path_param("/a/b.csv", "path_param") == "/a/b.csv"
assert check.path_param(Path("/a/b.csv"), "path_param") == "/a/b.csv"

with pytest.raises(ParameterCheckError):
check.path_param(None, "path_param")

with pytest.raises(ParameterCheckError):
check.path_param(0, "path_param")


def test_opt_path_param():
from pathlib import Path

assert check.opt_path_param("/a/b.csv", "path_param") == "/a/b.csv"
assert check.opt_path_param(Path("/a/b.csv"), "path_param") == "/a/b.csv"
assert check.opt_path_param(None, "path_param") is None
assert check.opt_path_param(None, "path_param", "/a/b/c.csv") == "/a/b/c.csv"
assert check.opt_path_param(None, "path_param", Path("/a/b/c.csv")) == "/a/b/c.csv"

with pytest.raises(ParameterCheckError):
check.opt_path_param(0, "path_param")


def test_bool_param():
assert check.bool_param(True, "b") is True
assert check.bool_param(False, "b") is False
Expand Down

0 comments on commit 02679a1

Please sign in to comment.