You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ib...@apache.org on 2019/10/31 23:59:45 UTC

[beam] branch master updated: [BEAM-8472] get default GCP region option (Python)

This is an automated email from the ASF dual-hosted git repository.

ibzib pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 6d5ca75  [BEAM-8472] get default GCP region option (Python)
     new 5030899  Merge pull request #9868 from ibzib/default-region
6d5ca75 is described below

commit 6d5ca757156a6b5a98303506963807e3e4f80ef6
Author: Kyle Weaver <kc...@google.com>
AuthorDate: Wed Oct 23 02:18:59 2019 +0200

    [BEAM-8472] get default GCP region option (Python)
---
 .../python/apache_beam/options/pipeline_options.py | 40 +++++++++++++++++-----
 1 file changed, 32 insertions(+), 8 deletions(-)

diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py
index 22d1586..d723694 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -22,6 +22,7 @@ from __future__ import absolute_import
 import argparse
 import json
 import logging
+import os
 from builtins import list
 from builtins import object
 
@@ -29,6 +30,7 @@ from apache_beam.options.value_provider import RuntimeValueProvider
 from apache_beam.options.value_provider import StaticValueProvider
 from apache_beam.options.value_provider import ValueProvider
 from apache_beam.transforms.display import HasDisplayData
+from apache_beam.utils import processes
 
 __all__ = [
     'PipelineOptions',
@@ -500,6 +502,32 @@ class GoogleCloudOptions(PipelineOptions):
                         choices=['COST_OPTIMIZED', 'SPEED_OPTIMIZED'],
                         help='Set the Flexible Resource Scheduling mode')
 
+  def _get_default_gcp_region(self):
+    """Get a default value for Google Cloud region according to
+    https://cloud.google.com/compute/docs/gcloud-compute/#default-properties.
+    If no other default can be found, returns 'us-central1'.
+    """
+    environment_region = os.environ.get('CLOUDSDK_COMPUTE_REGION')
+    if environment_region:
+      logging.info('Using default GCP region %s from $CLOUDSDK_COMPUTE_REGION',
+                   environment_region)
+      return environment_region
+    try:
+      cmd = ['gcloud', 'config', 'get-value', 'compute/region']
+      output = processes.check_output(cmd).decode('utf-8').strip()
+      if output:
+        logging.info('Using default GCP region %s from `%s`',
+                     output, ' '.join(cmd))
+        return output
+    except RuntimeError:
+      pass
+    logging.warning(
+        '--region not set; will default to us-central1. Future releases of '
+        'Beam will require the user to set --region explicitly, or else have a '
+        'default set via the gcloud tool. '
+        'https://cloud.google.com/compute/docs/regions-zones')
+    return 'us-central1'
+
   def validate(self, validator):
     errors = []
     if validator.is_service_runner():
@@ -514,14 +542,10 @@ class GoogleCloudOptions(PipelineOptions):
         errors.append('--dataflow_job_file and --template_location '
                       'are mutually exclusive.')
 
-    if self.view_as(GoogleCloudOptions).region is None:
-      self.view_as(GoogleCloudOptions).region = 'us-central1'
-      runner = self.view_as(StandardOptions).runner
-      if runner == 'DataflowRunner' or runner == 'TestDataflowRunner':
-        logging.warning(
-            '--region not set; will default to us-central1. Future releases of '
-            'Beam will require the user to set the region explicitly. '
-            'https://cloud.google.com/compute/docs/regions-zones/regions-zones')
+    runner = self.view_as(StandardOptions).runner
+    if runner == 'DataflowRunner' or runner == 'TestDataflowRunner':
+      if self.view_as(GoogleCloudOptions).region is None:
+        self.view_as(GoogleCloudOptions).region = self._get_default_gcp_region()
 
     return errors