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


##########
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:
   I think if the error is clear enough that's fine for now. Perhaps we could 
say `The provided equalities do not connect all of {inputs}.`



##########
sdks/python/apache_beam/yaml/yaml_join.py:
##########
@@ -0,0 +1,161 @@
+#
+# 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."""
+import networkx as nx
+import apache_beam as beam
+from apache_beam.yaml import yaml_provider
+
+
+def _validate_type(type, str):
+  error = ValueError('Invalid value for \'type\' in Join transform')
+  print(type)
+  if not isinstance(type, dict) and isinstance(type, str) != str:
+    return error
+  if isinstance(type, dict) and len(type) != 1 and next(
+      iter(type)) != 'outer' and not isinstance(type['outer'],
+                                                list):
+    return error
+  if isinstance(type, str) and type not in (
+      'inner', 'outer', 'left', 'right'):
+    return error
+
+
+def _validate_equalities(equalities, pcoll):
+  error = ValueError(
+    'Invalid value for \'equalities\' in Join transform')
+  if not isinstance(equalities, list):
+    return error
+  input_edge_list = []
+  for equality in equalities:
+    if len(equality) != 2 and not isinstance(equality, dict):
+      return error
+
+    for input, col in equality.items():
+      # TODO: look for an easier way to get field names in a Pcollection obj
+      possible_cols = set()
+      for field in pcoll['f'].element_type._fields:
+        possible_cols.add(field[0])
+      if input not in pcoll.keys() or col not in possible_cols:
+        return ValueError(
+          'Invalid input alias or column name doesn\'t exist in the input')
+
+    input_edge_list.append(tuple(equality.keys()))
+
+  if not nx.is_connected(nx.Graph(input_edge_list)):
+    return ValueError(
+      'Inputs in equalities are not all connected'
+    )
+
+
+def _parse_fields(tables, fields):
+  # TODO(titodo) - consider taking all validations to a preprocessing fn
+  output_fields = []
+  named_columns = set()
+  for input, cols in fields.items():
+    if input not in tables:
+      return ValueError(f'invalid input {input}')
+    if isinstance(cols, list):
+      for col in cols:
+        if col in named_columns:
+          return ValueError(
+            f'same field name {col} specified more than once')
+        output_fields.append(f'{input}.{col} AS {col}')
+        named_columns.add(col)
+    elif isinstance(cols, dict):
+      for k, v in cols.items():
+        if k in named_columns:
+          return ValueError(
+            f'same field name {k} specified more than once')
+        output_fields.append(f'{input}.{v} AS {k}')
+        named_columns.add(k)
+    else:
+      return ValueError(f'invalid entry for fields')
+  for table in tables:
+    if table not in fields.keys():
+      output_fields.append(f'{table}.*')
+  return output_fields
+
+
+@beam.ptransform.ptransform_fn
+def _SqlJoinTransform(pcoll, sql_transform_constructor, type,
+                      equalities, fields=None):
+  _validate_type(type, str)
+  _validate_equalities(equalities, pcoll)
+
+  tables = list(pcoll)
+  outer = []
+  if isinstance(type, dict):
+    outer = type['outer']
+  base_table = tables[0]
+  conditioned = {base_table}
+
+  def generate_join_type(left, right):
+    if left in outer and right in outer:
+      return 'FULL'
+    if left in outer:
+      return 'LEFT'
+    if right in outer:
+      return 'RIGHT'
+    if not outer:
+      return type.upper()
+    return 'INNER'
+
+  prev_table = tables[0]
+  join_conditions = {}
+  for i in range(1, len(tables)):
+    curr_table = tables[i]
+    join_type = generate_join_type(prev_table, curr_table)
+    join_conditions[curr_table] = f' {join_type} JOIN {curr_table}'
+    prev_table = curr_table
+
+  for equality in equalities:
+    keys = list(equality)
+    left, right = keys[0], keys[1]
+    # TODO(titodo) - shorten this and reduce repitition
+    if left in conditioned and right in conditioned:
+      t = tables[max(tables.index(left), tables.index(right))]

Review Comment:
   OK, that makes sense.



-- 
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