[
https://issues.apache.org/jira/browse/BEAM-2085?focusedWorklogId=582852&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-582852
]
ASF GitHub Bot logged work on BEAM-2085:
----------------------------------------
Author: ASF GitHub Bot
Created on: 14/Apr/21 20:18
Start Date: 14/Apr/21 20:18
Worklog Time Spent: 10m
Work Description: tvalentyn commented on a change in pull request #14390:
URL: https://github.com/apache/beam/pull/14390#discussion_r613559515
##########
File path: sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
##########
@@ -863,6 +863,31 @@ def test_pack_combiners_enabled_by_experiment(self):
self._test_pack_combiners(
PipelineOptions(self.default_properties), expect_packed=True)
+ def test_resource_hints_translation(self):
+ runner = DataflowRunner()
+
+ self.default_properties.append('--experiments=use_legacy_bq_sink')
+ self.default_properties.append('--resource_hint=min_ram_per_vcpu=10GB')
+ with beam.Pipeline(runner=runner,
+ options=PipelineOptions(self.default_properties)) as p:
+ # pylint: disable=expression-not-assigned
+ (
+ p
+ | beam.Create([1])
+ | 'MapWithHints' >> beam.Map(lambda x: x + 1).with_resource_hints(
+ min_ram_per_vcpu='20GB',
+
accelerator='type:nvidia-tesla-k80;count:1;install-nvidia-drivers'
+ ))
+
+ step = self._find_step(runner.job, 'MapWithHints')
+ self.assertEqual(
+ step['properties']['resource_hints'],
+ {
+ 'beam:resources:min_ram_per_vcpu_bytes:v1': '10000000000',
Review comment:
Discussed offline, keeping as is for now.
##########
File path: sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
##########
@@ -863,6 +863,31 @@ def test_pack_combiners_enabled_by_experiment(self):
self._test_pack_combiners(
PipelineOptions(self.default_properties), expect_packed=True)
+ def test_resource_hints_translation(self):
+ runner = DataflowRunner()
+
+ self.default_properties.append('--experiments=use_legacy_bq_sink')
+ self.default_properties.append('--resource_hint=min_ram_per_vcpu=10GB')
Review comment:
Done
##########
File path: sdks/python/apache_beam/runners/pipeline_context.py
##########
@@ -274,5 +282,44 @@ def to_runner_api(self):
return context_proto
def default_environment_id(self):
- # type: () -> Optional[str]
+ # type: () -> str
return self._default_environment_id
+
+ def get_environment_id_for_transform(
+ self, transform): # type: (Optional[ptransform.PTransform]) -> str
+ """Returns an environment id where the transform can be executed."""
+ if not transform or not transform.get_resource_hints():
+ return self.default_environment_id()
+
+ def merge_resource_hints(
+ environment_id,
+ transform): # type: (str, ptransform.PTransform) -> Dict[str, bytes]
+ # TODO: add test.
+ # Hints already defined in the environment take precedence over hints
Review comment:
Done.
##########
File path: sdks/python/apache_beam/transforms/resources.py
##########
@@ -0,0 +1,128 @@
+# 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.
+#
+
+"""A module for defining resource requirements for execution of transforms.
+
+Pipeline authors can use resource hints to provide additional information to
+runners about the desired aspects of the execution environment.
+
+Resource hints can be specified on a transform level for parts of the pipeline,
+or globally via --resource_hint pipeline option.
+
+See also: PTransforms.with_resource_hints().
+"""
+
+from typing import Any
+from typing import Callable
+from typing import Dict
+
+from apache_beam.portability.common_urns import resource_hints
+
+__all__ = ['parse_resource_hints', 'get_merged_hint_value']
+
+
+def _parse_str(value):
+ if not isinstance(value, str):
+ raise ValueError()
+ return value.encode('ascii')
+
+
+def _parse_int(value):
+ if isinstance(value, str):
+ value = int(value)
+ if not isinstance(value, int):
+ raise ValueError()
+ return str(value).encode('ascii')
+
+
+def _parse_any(_):
+ # For hints where only a key is relevant and value is set to None or any
value
+ return b'1'
+
+
+def _parse_storage_size_str(value): # type: (str) -> bytes
+ """Parses a human-friendly storage size string into a number of bytes.
+ """
+ if not isinstance(value, str):
+ value = str(value)
+ value = value.strip().replace(" ", "")
+ units = {
+ 'PiB': 2**50,
+ 'TiB': 2**40,
+ 'GiB': 2**30,
+ 'MiB': 2**20,
+ 'KiB': 2**10,
+ 'PB': 10**15,
+ 'TB': 10**12,
+ 'GB': 10**9,
+ 'MB': 10**6,
+ 'KB': 10**3,
+ }
+ multiplier = 1
+ for suffix in units:
+ if value.endswith(suffix):
+ multiplier = units[suffix]
+ value = value[:-len(suffix)]
+ break
+
+ return str(round(float(value) * multiplier)).encode('ascii')
+
+
+def _use_max(v1, v2):
Review comment:
Done
--
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 582852)
Time Spent: 12h 20m (was: 12h 10m)
> Transforms could give hints to runners on resource requirements
> ---------------------------------------------------------------
>
> Key: BEAM-2085
> URL: https://issues.apache.org/jira/browse/BEAM-2085
> Project: Beam
> Issue Type: Improvement
> Components: beam-model, runner-core, sdk-java-core
> Affects Versions: Not applicable
> Reporter: Ismaël Mejía
> Assignee: Valentyn Tymofieiev
> Priority: P3
> Labels: Clarified
> Time Spent: 12h 20m
> Remaining Estimate: 0h
>
> As discussed in BEAM-673 runners can allocate workers to accomplish their
> work in a better way if they can take into account some hints from the
> transforms, e.g. a source can hint data locality and with this information
> the runner can allocate the workers in an better way, this can also be the
> case with a particular transform that can suggest to the runner to be
> executed in a worker with a specific resource, e.g. GPU.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)