itodotimothy6 commented on code in PR #30734:
URL: https://github.com/apache/beam/pull/30734#discussion_r1557157650


##########
sdks/python/apache_beam/yaml/yaml_join.py:
##########
@@ -0,0 +1,273 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""This module defines the Join operation."""
+from typing import Any
+from typing import Dict
+from typing import List
+from typing import Optional
+from typing import Union
+
+import apache_beam as beam
+from apache_beam.yaml import yaml_provider
+
+
+def _validate_input(pcolls):
+  error_prefix = f'Invalid input {pcolls} specified.'
+  if not isinstance(pcolls, dict):
+    raise ValueError(f'{error_prefix} It must be a dict.')
+  if len(pcolls) < 2:
+    raise ValueError(
+        f'{error_prefix} There should be at least 2 inputs to join.')
+
+
+def _validate_type(type):
+  error_prefix = f'Invalid value "{type}" for "type". It should be'
+  if not isinstance(type, dict) and not isinstance(type, str):
+    raise ValueError(f'{error_prefix} a dict or a str.')
+  if isinstance(type, dict):
+    error = ValueError(
+        f'{error_prefix} a dictionary of type [str, list[str]] '
+        f'and have only one element with the key "outer".')
+    if len(type) != 1:
+      raise error
+    if next(iter(type)) != 'outer':
+      raise error
+    if not isinstance(type['outer'], list):
+      raise error
+  if isinstance(type, str) and type not in ('inner', 'outer', 'left', 'right'):
+    raise ValueError(
+        f'{error_prefix} '
+        f'one of the following: "inner", "outer", "left", "right"')
+
+
+def _validate_equalities(equalities, pcolls):
+  error_prefix = f'Invalid value "{equalities}" for "equalities".'
+
+  valid_cols = {}
+  inputs = list(pcolls.keys())
+  for input in inputs:
+    fields = set()
+    for field in pcolls[input].element_type._fields:
+      fields.add(field[0])
+    valid_cols[input] = fields
+
+  if isinstance(equalities, str):
+    for input in valid_cols.keys():
+      if equalities not in valid_cols[input]:
+        raise ValueError(
+            f'{error_prefix} When "equalities" is a str, '
+            f'it must be a field name that exists in all the specified 
inputs.')
+    equality = {input: equalities for input in inputs}
+    return [equality]
+
+  if not isinstance(equalities, list):
+    raise ValueError(f'{error_prefix} It should be a str or a list.')
+
+  input_edge_list = []
+  for equality in equalities:
+    invalid_dict_error = ValueError(
+        f'{error_prefix} {equality} '
+        f'should be a dict[str, str] containing at least 2 items.')
+    if not isinstance(equality, dict):
+      raise invalid_dict_error
+    if len(equality) < 2:
+      raise invalid_dict_error
+
+    for input, col in equality.items():
+      if input not in inputs:
+        raise ValueError(
+            f'{error_prefix} "{input}" is not a specified alias in "input"')
+      if col not in valid_cols[input]:
+        raise ValueError(
+            f'{error_prefix} "{col}" is not a valid field in "{input}"')
+
+    input_edge_list.append(tuple(equality.keys()))
+
+  if not _is_connected(input_edge_list):
+    raise ValueError(

Review Comment:
   Since we're not supporting cross join (yet), if all the inputs are not 
connected by `equalities`, a ValueError will be raised, indicating to the user 
to ensure their equalities connect all the inputs in their transform.
   
   Thoughts? @Polber @robertwb 



##########
sdks/python/apache_beam/yaml/yaml_join_test.py:
##########
@@ -0,0 +1,210 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import logging
+import unittest
+
+import apache_beam as beam
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+from apache_beam.yaml.yaml_transform import YamlTransform
+
+
+class ToRow(beam.PTransform):
+  def expand(self, pcoll):
+    return pcoll | beam.Map(lambda row: beam.Row(**row._asdict()))
+
+
+FRUITS = [
+    beam.Row(id=1, name='raspberry'),
+    beam.Row(id=2, name='blackberry'),
+]
+
+QUANTITIES = [
+    beam.Row(name='raspberry', quantity=1),
+    beam.Row(name='blackberry', quantity=2),
+    beam.Row(name='blueberry', quantity=3),
+]
+
+CATEGORIES = [
+    beam.Row(name='raspberry', category='juicy'),
+    beam.Row(name='blackberry', category='dry'),
+    beam.Row(name='blueberry', category='dry'),
+    beam.Row(name='blueberry', category='juicy'),
+]
+
+
+class YamlJoinTest(unittest.TestCase):

Review Comment:
   Thanks! Added. The pre-commits are now passing.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscr...@beam.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to