KevinGG commented on a change in pull request #16741:
URL: https://github.com/apache/beam/pull/16741#discussion_r808575723
##########
File path: sdks/python/apache_beam/runners/interactive/interactive_runner.py
##########
@@ -209,6 +218,73 @@ 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 not ie.current_env(). \
+ clusters.dataproc_cluster_managers.get(
+ str(id(user_pipeline)), 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] \
Review comment:
Ditto, these can be linted as
```
clusters.master_urls[
cluster_manager.master_url] = ...
...
```
Normally, a linebreak is not needed.
##########
File path: sdks/python/apache_beam/runners/interactive/interactive_runner.py
##########
@@ -209,6 +218,73 @@ 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 not ie.current_env(). \
Review comment:
Can you utilize the linter to format this instead of using a linebreak?
##########
File path: sdks/python/apache_beam/runners/interactive/interactive_runner.py
##########
@@ -209,6 +218,73 @@ 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 not ie.current_env(). \
+ clusters.dataproc_cluster_managers.get(
+ str(id(user_pipeline)), 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)))
+ else:
+ _LOGGER.info(
Review comment:
This else branch is not needed.
##########
File path:
sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py
##########
@@ -84,46 +89,71 @@ 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)
+ self.master_url = self.get_master_url(self.cluster_metadata)
except Exception as e:
if e.code == 409:
- if self._cluster_name == self.DEFAULT_NAME:
+ if self.cluster_metadata in ie.current_env(
+ ).clusters.master_urls.inverse:
Review comment:
This can be moved to the constructor. And then, in this function, check
if self.master_url exists before calling the dataproc client. Essentially,
disassembling the get_master_url function, only keep the TODO part in it.
Keep the 409 execution route with the info logging only. Practically, this
execution route will be a branch that never gets executed if the dataproc
cluster manger is created and used correctly.
--
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]