KevinGG commented on a change in pull request #16741:
URL: https://github.com/apache/beam/pull/16741#discussion_r809372731
##########
File path:
sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py
##########
@@ -84,46 +95,58 @@ def create_cluster(self, cluster: dict) -> None:
try:
Review comment:
Before the try, you can add
```
if self.master_url:
return
```
##########
File path:
sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py
##########
@@ -18,60 +18,71 @@
# pytype: skip-file
import logging
+from dataclasses import dataclass
from typing import Optional
-try:
- from google.cloud import dataproc_v1
-except ImportError:
- raise ImportError(
- 'Google Cloud Dataproc not supported for this execution environment.')
+from apache_beam.runners.interactive import interactive_environment as ie
_LOGGER = logging.getLogger(__name__)
+@dataclass
+class MasterURLIdentifier:
+ project_id: Optional[str] = None
+ region: Optional[str] = None
+ cluster_name: Optional[str] = None
+
+ def __key(self):
+ return (self.project_id, self.region, self.cluster_name)
+
+ def __hash__(self):
+ return hash(self.__key())
+
+ def __eq__(self, other):
+ if isinstance(other, MasterURLIdentifier):
+ return self.__key() == other.__key()
+ raise NotImplementedError(
+ 'Comparisons are only supported between '
+ 'instances of MasterURLIdentifier.')
+
+
class DataprocClusterManager:
"""The DataprocClusterManager object simplifies the operations
required for creating and deleting Dataproc clusters for use
under Interactive Beam.
"""
- DEFAULT_NAME = 'interactive-beam-cluster'
-
- def __init__(
- self,
- project_id: Optional[str] = None,
- region: Optional[str] = None,
- cluster_name: Optional[str] = None) -> None:
+ def __init__(self, cluster_metadata: MasterURLIdentifier) -> None:
"""Initializes the DataprocClusterManager with properties required
to interface with the Dataproc ClusterControllerClient.
"""
-
- self._project_id = project_id
- if region == 'global':
+ self.cluster_metadata = cluster_metadata
+ if self.cluster_metadata.region == 'global':
# The global region is unsupported as it will be eventually deprecated.
raise ValueError('Clusters in the global region are not supported.')
- elif region:
- self._region = region
- else:
+ elif not self.cluster_metadata.region:
_LOGGER.warning(
'No region information was detected, defaulting Dataproc cluster '
'region to: us-central1.')
- self._region = 'us-central1'
+ self.cluster_metadata.region = 'us-central1'
- if cluster_name:
- _LOGGER.warning(
- 'A user-specified cluster_name has been detected. '
- 'Please note that you will have to manually delete the Dataproc '
- 'cluster that will be created under the name: %s',
- cluster_name)
- self._cluster_name = cluster_name
- else:
- self._cluster_name = self.DEFAULT_NAME
+ if not self.cluster_metadata.cluster_name:
+ self.cluster_metadata.cluster_name = ie.current_env(
+ ).clusters.default_cluster_name
+ from google.cloud import dataproc_v1
self._cluster_client = dataproc_v1.ClusterControllerClient(
client_options={
- 'api_endpoint': f'{self._region}-dataproc.googleapis.com:443'
+ 'api_endpoint':
f'{self.cluster_metadata.region}-dataproc.googleapis.com:443'
})
+ if self.cluster_metadata in ie.current_env().clusters.master_urls.inverse:
+ self.master_url = self.get_master_url(self.cluster_metadata)
Review comment:
This can be just self.master_url =
ie.current_env().clusters.master_urls.inverse[self.cluster_metadata],
And no need to log a warning about it.
##########
File path:
sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py
##########
@@ -84,46 +95,58 @@ def create_cluster(self, cluster: dict) -> None:
try:
self._cluster_client.create_cluster(
request={
- 'project_id': self._project_id,
- 'region': self._region,
+ 'project_id': self.cluster_metadata.project_id,
+ 'region': self.cluster_metadata.region,
'cluster': cluster
})
- _LOGGER.info('Cluster created successfully: %s', self._cluster_name)
+ _LOGGER.info(
+ 'Cluster created successfully: %s',
+ self.cluster_metadata.cluster_name)
+ if not self.master_url:
Review comment:
This if-statement is not needed because master_url is definitely not
assigned.
##########
File path: sdks/python/apache_beam/runners/interactive/interactive_runner.py
##########
@@ -209,6 +218,70 @@ def visit_transform(self, transform_node):
return main_job_result
+ # TODO(victorhc): Move this method somewhere else if performance is impacted
+ # by generating a cluster during runtime.
+ def _create_dataproc_cluster_if_applicable(self, user_pipeline):
+ """ Creates a Dataproc cluster if the provided user_pipeline is running
+ FlinkRunner and no flink_master_url was provided as an option. A cluster
+ is not created when a flink_master_url is detected.
+
+ Example pipeline options to enable automatic Dataproc cluster creation:
+ options = PipelineOptions([
+ '--runner=FlinkRunner',
+ '--project=my-project',
+ '--region=my-region',
+ '--environment_type=DOCKER'
+ ])
+
+ Example pipeline options to skip automatic Dataproc cluster creation:
+ options = PipelineOptions([
+ '--runner=FlinkRunner',
+ '--flink_master=example.internal:41979',
+ '--environment_type=DOCKER'
+ ])
+ """
+ from apache_beam.runners.portability.flink_runner import FlinkRunner
+ from apache_beam.options.pipeline_options import FlinkRunnerOptions
+ flink_master = user_pipeline.options.view_as(
+ FlinkRunnerOptions).flink_master
+ clusters = ie.current_env().clusters
+ # Only consider this logic when both below 2 conditions apply.
+ if isinstance(self._underlying_runner,
+ FlinkRunner) and clusters.dataproc_cluster_managers.get(
+ str(id(user_pipeline)), None) is None:
+ if flink_master == '[auto]':
+ # The above condition is True when the user has not provided a
+ # flink_master.
+ if ie.current_env()._is_in_ipython:
+ warnings.filterwarnings(
+ 'ignore',
+ 'options is deprecated since First stable release. References to
'
+ '<pipeline>.options will not be supported',
+ category=DeprecationWarning)
+ project_id =
(user_pipeline.options.view_as(GoogleCloudOptions).project)
+ region = (user_pipeline.options.view_as(GoogleCloudOptions).region)
+ cluster_name = ie.current_env().clusters.default_cluster_name
+ cluster_metadata = MasterURLIdentifier(
+ project_id=project_id, region=region, cluster_name=cluster_name)
+ elif flink_master in clusters.master_urls:
+ cluster_metadata = clusters.master_urls.inverse.get(flink_master, None)
+ # else noop, no need to log anything because we allow a master_url
+ # (not managed by us) provided by the user.
+ if cluster_metadata:
+ # create the cluster_manager and populate dicts in the clusters
+ # instance if the pipeline is not already mapped to an existing
+ # cluster_manager.
+ cluster_manager = DataprocClusterManager(cluster_metadata)
+ cluster_manager.create_flink_cluster()
+ clusters.master_urls[cluster_manager.master_url] = cluster_metadata
+ clusters.dataproc_cluster_managers[str(
+ id(user_pipeline))] = cluster_manager
+ clusters.master_urls_to_pipelines[cluster_manager.master_url].append(
+ str(id(user_pipeline)))
+ _LOGGER.info(
Review comment:
This logging is not needed.
--
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]