Skip to content

Commit 8b1ca21

Browse files
authored
Merge pull request #32776 Propogate field_descriptions to RowTypeConstraint
2 parents a0cbe74 + 80c7450 commit 8b1ca21

File tree

3 files changed

+54
-5
lines changed

3 files changed

+54
-5
lines changed

sdks/python/apache_beam/typehints/row_type.py

Lines changed: 15 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,8 @@ def __init__(
4949
fields: Sequence[Tuple[str, type]],
5050
user_type,
5151
schema_options: Optional[Sequence[Tuple[str, Any]]] = None,
52-
field_options: Optional[Dict[str, Sequence[Tuple[str, Any]]]] = None):
52+
field_options: Optional[Dict[str, Sequence[Tuple[str, Any]]]] = None,
53+
field_descriptions: Optional[Dict[str, str]] = None):
5354
"""For internal use only, no backwards comatibility guaratees. See
5455
https://beam.apache.org/documentation/programming-guide/#schemas-for-pl-types
5556
for guidance on creating PCollections with inferred schemas.
@@ -96,6 +97,7 @@ def __init__(
9697

9798
self._schema_options = schema_options or []
9899
self._field_options = field_options or {}
100+
self._field_descriptions = field_descriptions or {}
99101

100102
@staticmethod
101103
def from_user_type(
@@ -107,20 +109,24 @@ def from_user_type(
107109
fields = [(name, user_type.__annotations__[name])
108110
for name in user_type._fields]
109111

112+
field_descriptions = getattr(user_type, '_field_descriptions', None)
113+
110114
if _user_type_is_generated(user_type):
111115
return RowTypeConstraint.from_fields(
112116
fields,
113117
schema_id=getattr(user_type, _BEAM_SCHEMA_ID),
114118
schema_options=schema_options,
115-
field_options=field_options)
119+
field_options=field_options,
120+
field_descriptions=field_descriptions)
116121

117122
# TODO(https://github.com/apache/beam/issues/22125): Add user API for
118123
# specifying schema/field options
119124
return RowTypeConstraint(
120125
fields=fields,
121126
user_type=user_type,
122127
schema_options=schema_options,
123-
field_options=field_options)
128+
field_options=field_options,
129+
field_descriptions=field_descriptions)
124130

125131
return None
126132

@@ -131,13 +137,15 @@ def from_fields(
131137
schema_options: Optional[Sequence[Tuple[str, Any]]] = None,
132138
field_options: Optional[Dict[str, Sequence[Tuple[str, Any]]]] = None,
133139
schema_registry: Optional[SchemaTypeRegistry] = None,
140+
field_descriptions: Optional[Dict[str, str]] = None,
134141
) -> RowTypeConstraint:
135142
return GeneratedClassRowTypeConstraint(
136143
fields,
137144
schema_id=schema_id,
138145
schema_options=schema_options,
139146
field_options=field_options,
140-
schema_registry=schema_registry)
147+
schema_registry=schema_registry,
148+
field_descriptions=field_descriptions)
141149

142150
def __call__(self, *args, **kwargs):
143151
# We make RowTypeConstraint callable (defers to constructing the user type)
@@ -206,6 +214,7 @@ def __init__(
206214
schema_options: Optional[Sequence[Tuple[str, Any]]] = None,
207215
field_options: Optional[Dict[str, Sequence[Tuple[str, Any]]]] = None,
208216
schema_registry: Optional[SchemaTypeRegistry] = None,
217+
field_descriptions: Optional[Dict[str, str]] = None,
209218
):
210219
from apache_beam.typehints.schemas import named_fields_to_schema
211220
from apache_beam.typehints.schemas import named_tuple_from_schema
@@ -224,7 +233,8 @@ def __init__(
224233
fields,
225234
user_type,
226235
schema_options=schema_options,
227-
field_options=field_options)
236+
field_options=field_options,
237+
field_descriptions=field_descriptions)
228238

229239
def __reduce__(self):
230240
return (

sdks/python/apache_beam/typehints/schemas.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -274,6 +274,7 @@ def typing_to_runner_api(self, type_: type) -> schema_pb2.FieldType:
274274
self.option_to_runner_api(option_tuple)
275275
for option_tuple in type_.field_options(field_name)
276276
],
277+
description=type_._field_descriptions.get(field_name, None),
277278
) for (field_name, field_type) in type_._fields
278279
],
279280
id=schema_id,

sdks/python/apache_beam/typehints/schemas_test.py

Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -489,6 +489,44 @@ def test_row_type_constraint_to_schema_with_field_options(self):
489489
]
490490
self.assertEqual(list(field.options), expected)
491491

492+
def test_row_type_constraint_to_schema_with_field_descriptions(self):
493+
row_type_with_options = row_type.RowTypeConstraint.from_fields(
494+
[
495+
('foo', np.int8),
496+
('bar', float),
497+
('baz', bytes),
498+
],
499+
field_descriptions={
500+
'foo': 'foo description',
501+
'bar': 'bar description',
502+
'baz': 'baz description',
503+
})
504+
result_type = typing_to_runner_api(row_type_with_options)
505+
506+
self.assertIsInstance(result_type, schema_pb2.FieldType)
507+
self.assertEqual(result_type.WhichOneof("type_info"), "row_type")
508+
509+
fields = result_type.row_type.schema.fields
510+
511+
expected = [
512+
schema_pb2.Field(
513+
name='foo',
514+
description='foo description',
515+
type=schema_pb2.FieldType(atomic_type=schema_pb2.BYTE),
516+
),
517+
schema_pb2.Field(
518+
name='bar',
519+
description='bar description',
520+
type=schema_pb2.FieldType(atomic_type=schema_pb2.DOUBLE),
521+
),
522+
schema_pb2.Field(
523+
name='baz',
524+
description='baz description',
525+
type=schema_pb2.FieldType(atomic_type=schema_pb2.BYTES),
526+
),
527+
]
528+
self.assertEqual(list(fields), expected)
529+
492530
def assert_namedtuple_equivalent(self, actual, expected):
493531
# Two types are only considered equal if they are literally the same
494532
# object (i.e. `actual == expected` is the same as `actual is expected` in

0 commit comments

Comments
 (0)