chamikaramj commented on code in PR #31495: URL: https://github.com/apache/beam/pull/31495#discussion_r1681545886
########## sdks/python/apache_beam/transforms/managed.py: ########## @@ -0,0 +1,173 @@ +# +# 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. +# + +"""Managed Transforms. + +This module builds and instantiates turnkey transforms that can be managed by +the underlying runner. + +Using Managed Transforms +======================== +Managed transforms have a defined configuration and can be built using an +inline :class:`dict` like so:: + + results = p | beam.managed.Read( + beam.managed.ICEBERG, + config={"param_1": "foo", + "param_2": "bar"}) + +A YAML configuration file can also be used to build a Managed transform. Say we +have the following `config.yaml` file:: + + param_1: "foo" + param_2: "bar" + +Simply provide the location to the file like so:: + + input_rows = p | beam.Create(...) + input_rows | beam.managed.Write( + beam.managed.KAFKA, + config_url="path/to/config.yaml") + +Available transforms +==================== +Available transforms are: + +- **Kafka** +- **Iceberg** + +**Note:** inputs and outputs need to be PCollections of Beam +:py:class:`apache_beam.pvalue.Row` elements. + +**Note:** This Managed API uses Java's ManagedSchemaTransform under the hood. +""" + +from typing import Any +from typing import Dict +from typing import Optional + +import yaml + +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import SchemaAwareExternalTransform +from apache_beam.transforms.ptransform import PTransform + +ICEBERG = "iceberg" +KAFKA = "kafka" +_MANAGED_IDENTIFIER = "beam:transform:managed:v1" +_GRADLE_TARGETS = { + "sdks:java:io:expansion-service:shadowJar": [KAFKA, ICEBERG], +} + +__all__ = ["ICEBERG", "KAFKA", "Read", "Write"] + + +class Read(PTransform): + """Read using Managed Transforms""" + READ_TRANSFORMS = { + ICEBERG: "beam:schematransform:org.apache.beam:iceberg_read:v1", + KAFKA: "beam:schematransform:org.apache.beam:kafka_read:v1", + } + + def __init__( + self, + source: str, + config: Optional[Dict[str, Any]] = None, + config_url: Optional[str] = None, + expansion_service=None): + super().__init__() + self._source = source + identifier = self.READ_TRANSFORMS.get(source.lower()) + if not identifier: + raise ValueError( + f"An unsupported source was specified: '{source}'. Please specify " + f"one of the following sources: {self.READ_TRANSFORMS.keys()}") + + self._expansion_service = _resolve_expansion_service( + source, identifier, expansion_service) + self._underlying_identifier = identifier + self._yaml_config = yaml.dump(config) + self._config_url = config_url + + def expand(self, input): + return input | SchemaAwareExternalTransform( + identifier=_MANAGED_IDENTIFIER, + expansion_service=self._expansion_service, + rearrange_based_on_discovery=True, + transform_identifier=self._underlying_identifier, + config=self._yaml_config, Review Comment: Only one of config or config_url should be set, right ? Should we also validate here ? ########## sdks/python/apache_beam/transforms/managed.py: ########## @@ -0,0 +1,173 @@ +# +# 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. +# + +"""Managed Transforms. + +This module builds and instantiates turnkey transforms that can be managed by +the underlying runner. + +Using Managed Transforms +======================== +Managed transforms have a defined configuration and can be built using an +inline :class:`dict` like so:: + + results = p | beam.managed.Read( + beam.managed.ICEBERG, + config={"param_1": "foo", + "param_2": "bar"}) + +A YAML configuration file can also be used to build a Managed transform. Say we +have the following `config.yaml` file:: + + param_1: "foo" + param_2: "bar" + +Simply provide the location to the file like so:: + + input_rows = p | beam.Create(...) + input_rows | beam.managed.Write( + beam.managed.KAFKA, + config_url="path/to/config.yaml") + +Available transforms +==================== +Available transforms are: + +- **Kafka** +- **Iceberg** + +**Note:** inputs and outputs need to be PCollections of Beam +:py:class:`apache_beam.pvalue.Row` elements. + +**Note:** This Managed API uses Java's ManagedSchemaTransform under the hood. +""" + +from typing import Any +from typing import Dict +from typing import Optional + +import yaml + +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import SchemaAwareExternalTransform +from apache_beam.transforms.ptransform import PTransform + +ICEBERG = "iceberg" +KAFKA = "kafka" +_MANAGED_IDENTIFIER = "beam:transform:managed:v1" +_GRADLE_TARGETS = { Review Comment: `_EXPANSION_SERVICE_JAR_TARGETS` ? ########## sdks/python/apache_beam/transforms/managed.py: ########## @@ -0,0 +1,173 @@ +# +# 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. +# + +"""Managed Transforms. + +This module builds and instantiates turnkey transforms that can be managed by +the underlying runner. + +Using Managed Transforms +======================== +Managed transforms have a defined configuration and can be built using an +inline :class:`dict` like so:: + + results = p | beam.managed.Read( + beam.managed.ICEBERG, Review Comment: (1) It can be any transform. (2) API wise, I think it's cleaner to separate these. Basically what the user is saying is, "I want to use the Managed Iceberg transform and use this config to configure it". The config can be available locally or remotely. Also, this make it easy to identify the primary function of a transform looking at a pipeline definition. ########## sdks/python/apache_beam/transforms/managed.py: ########## @@ -0,0 +1,173 @@ +# +# 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. +# + +"""Managed Transforms. + +This module builds and instantiates turnkey transforms that can be managed by Review Comment: Also, let's make sure that this is consistent with (and has the same details as) what we mention for Java: https://github.com/apache/beam/blob/2d6d55b98cef5eacc4abd93863de2132cf89bea2/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java#L4 Specifically what it means by "managed by the runner" is up to the runner. Some runners may do absolutely nothing while other runners may upgrade transforms to more recent/better versions. ########## sdks/python/apache_beam/transforms/managed.py: ########## @@ -0,0 +1,173 @@ +# +# 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. +# + +"""Managed Transforms. + +This module builds and instantiates turnkey transforms that can be managed by +the underlying runner. + +Using Managed Transforms +======================== +Managed transforms have a defined configuration and can be built using an +inline :class:`dict` like so:: + + results = p | beam.managed.Read( + beam.managed.ICEBERG, + config={"param_1": "foo", + "param_2": "bar"}) + +A YAML configuration file can also be used to build a Managed transform. Say we +have the following `config.yaml` file:: + + param_1: "foo" + param_2: "bar" + +Simply provide the location to the file like so:: + + input_rows = p | beam.Create(...) + input_rows | beam.managed.Write( + beam.managed.KAFKA, + config_url="path/to/config.yaml") + +Available transforms +==================== +Available transforms are: + +- **Kafka** +- **Iceberg** + +**Note:** inputs and outputs need to be PCollections of Beam Review Comment: \`PCollection\` (also for other class names used here) ########## sdks/python/apache_beam/transforms/managed.py: ########## @@ -0,0 +1,173 @@ +# +# 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. +# + +"""Managed Transforms. + +This module builds and instantiates turnkey transforms that can be managed by +the underlying runner. + +Using Managed Transforms +======================== +Managed transforms have a defined configuration and can be built using an +inline :class:`dict` like so:: + + results = p | beam.managed.Read( + beam.managed.ICEBERG, + config={"param_1": "foo", + "param_2": "bar"}) + +A YAML configuration file can also be used to build a Managed transform. Say we +have the following `config.yaml` file:: + + param_1: "foo" + param_2: "bar" + +Simply provide the location to the file like so:: + + input_rows = p | beam.Create(...) + input_rows | beam.managed.Write( + beam.managed.KAFKA, + config_url="path/to/config.yaml") + +Available transforms +==================== +Available transforms are: + +- **Kafka** +- **Iceberg** + +**Note:** inputs and outputs need to be PCollections of Beam +:py:class:`apache_beam.pvalue.Row` elements. + +**Note:** This Managed API uses Java's ManagedSchemaTransform under the hood. +""" + +from typing import Any +from typing import Dict +from typing import Optional + +import yaml + +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import SchemaAwareExternalTransform +from apache_beam.transforms.ptransform import PTransform + +ICEBERG = "iceberg" +KAFKA = "kafka" +_MANAGED_IDENTIFIER = "beam:transform:managed:v1" +_GRADLE_TARGETS = { + "sdks:java:io:expansion-service:shadowJar": [KAFKA, ICEBERG], +} + +__all__ = ["ICEBERG", "KAFKA", "Read", "Write"] + + +class Read(PTransform): + """Read using Managed Transforms""" + READ_TRANSFORMS = { + ICEBERG: "beam:schematransform:org.apache.beam:iceberg_read:v1", Review Comment: +1 (also for Java). ########## sdks/python/apache_beam/transforms/managed.py: ########## @@ -0,0 +1,173 @@ +# +# 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. +# + +"""Managed Transforms. + +This module builds and instantiates turnkey transforms that can be managed by +the underlying runner. + +Using Managed Transforms +======================== +Managed transforms have a defined configuration and can be built using an +inline :class:`dict` like so:: + + results = p | beam.managed.Read( + beam.managed.ICEBERG, + config={"param_1": "foo", + "param_2": "bar"}) + +A YAML configuration file can also be used to build a Managed transform. Say we +have the following `config.yaml` file:: + + param_1: "foo" + param_2: "bar" + +Simply provide the location to the file like so:: + + input_rows = p | beam.Create(...) + input_rows | beam.managed.Write( + beam.managed.KAFKA, + config_url="path/to/config.yaml") + +Available transforms +==================== +Available transforms are: + +- **Kafka** +- **Iceberg** + +**Note:** inputs and outputs need to be PCollections of Beam +:py:class:`apache_beam.pvalue.Row` elements. + +**Note:** This Managed API uses Java's ManagedSchemaTransform under the hood. +""" + +from typing import Any +from typing import Dict +from typing import Optional + +import yaml + +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import SchemaAwareExternalTransform +from apache_beam.transforms.ptransform import PTransform + +ICEBERG = "iceberg" +KAFKA = "kafka" +_MANAGED_IDENTIFIER = "beam:transform:managed:v1" +_GRADLE_TARGETS = { + "sdks:java:io:expansion-service:shadowJar": [KAFKA, ICEBERG], +} + +__all__ = ["ICEBERG", "KAFKA", "Read", "Write"] + + +class Read(PTransform): + """Read using Managed Transforms""" + READ_TRANSFORMS = { + ICEBERG: "beam:schematransform:org.apache.beam:iceberg_read:v1", + KAFKA: "beam:schematransform:org.apache.beam:kafka_read:v1", + } + + def __init__( + self, + source: str, + config: Optional[Dict[str, Any]] = None, + config_url: Optional[str] = None, + expansion_service=None): + super().__init__() + self._source = source + identifier = self.READ_TRANSFORMS.get(source.lower()) + if not identifier: + raise ValueError( + f"An unsupported source was specified: '{source}'. Please specify " + f"one of the following sources: {self.READ_TRANSFORMS.keys()}") + + self._expansion_service = _resolve_expansion_service( + source, identifier, expansion_service) + self._underlying_identifier = identifier + self._yaml_config = yaml.dump(config) + self._config_url = config_url + + def expand(self, input): + return input | SchemaAwareExternalTransform( + identifier=_MANAGED_IDENTIFIER, + expansion_service=self._expansion_service, + rearrange_based_on_discovery=True, + transform_identifier=self._underlying_identifier, + config=self._yaml_config, + config_url=self._config_url) + + def default_label(self) -> str: + return "Managed Read(%s)" % self._source.upper() + + +class Write(PTransform): + """Write using Managed Transforms""" + WRITE_TRANSFORMS = { + ICEBERG: "beam:schematransform:org.apache.beam:iceberg_write:v1", + KAFKA: "beam:schematransform:org.apache.beam:kafka_write:v1", + } + + def __init__( + self, + sink: str, + config: Optional[Dict[str, Any]] = None, + config_url: Optional[str] = None, + expansion_service=None): + super().__init__() + self._sink = sink + identifier = self.WRITE_TRANSFORMS.get(sink.lower()) + if not identifier: + raise ValueError( + f"An unsupported sink was specified: '{sink}'. Please specify " + f"one of the following sinks: {self.WRITE_TRANSFORMS.keys()}") + + self._expansion_service = _resolve_expansion_service( + sink, identifier, expansion_service) + self._underlying_identifier = identifier + self._yaml_config = yaml.dump(config) + self._config_url = config_url + + def expand(self, input): Review Comment: Can we add some unit-test coverage using simple pipelines ? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
