aaltay commented on a change in pull request #20386:
URL: https://github.com/apache/airflow/pull/20386#discussion_r773601098
##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -185,56 +289,30 @@ def __init__(
dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
**kwargs,
) -> None:
- super().__init__(**kwargs)
+ super().__init__(
+ runner=runner,
+ default_pipeline_options=default_pipeline_options,
+ pipeline_options=pipeline_options,
+ gcp_conn_id=gcp_conn_id,
+ delegate_to=delegate_to,
+ dataflow_config=dataflow_config,
+ **kwargs,
+ )
self.py_file = py_file
- self.runner = runner
self.py_options = py_options or []
- self.default_pipeline_options = default_pipeline_options or {}
- self.pipeline_options = pipeline_options or {}
- self.pipeline_options.setdefault("labels", {}).update(
- {"airflow-version": "v" + version.replace(".", "-").replace("+",
"-")}
- )
self.py_interpreter = py_interpreter
self.py_requirements = py_requirements
self.py_system_site_packages = py_system_site_packages
- self.gcp_conn_id = gcp_conn_id
- self.delegate_to = delegate_to
- self.beam_hook: Optional[BeamHook] = None
- self.dataflow_hook: Optional[DataflowHook] = None
- self.dataflow_job_id: Optional[str] = None
-
- if dataflow_config is None:
- self.dataflow_config = DataflowConfiguration()
- elif isinstance(dataflow_config, dict):
- self.dataflow_config = DataflowConfiguration(**dataflow_config)
- else:
- self.dataflow_config = dataflow_config
-
- if self.dataflow_config and self.runner.lower() !=
BeamRunnerType.DataflowRunner.lower():
- self.log.warning(
- "dataflow_config is defined but runner is different than
DataflowRunner (%s)", self.runner
- )
def execute(self, context):
"""Execute the Apache Beam Pipeline."""
- self.beam_hook = BeamHook(runner=self.runner)
- pipeline_options = self.default_pipeline_options.copy()
- process_line_callback: Optional[Callable] = None
- is_dataflow = self.runner.lower() ==
BeamRunnerType.DataflowRunner.lower()
- dataflow_job_name: Optional[str] = None
-
- if is_dataflow:
- dataflow_job_name, pipeline_options, process_line_callback =
self._set_dataflow(
- pipeline_options=pipeline_options,
job_name_variable_key="job_name"
- )
-
- pipeline_options.update(self.pipeline_options)
-
- # Convert argument names from lowerCamelCase to snake case.
- formatted_pipeline_options = {
- convert_camel_to_snake(key): pipeline_options[key] for key in
pipeline_options
- }
+ (
+ is_dataflow,
+ dataflow_job_name,
+ formatted_pipeline_options,
+ process_line_callback,
+ ) = self._init_pipeline_options(format_pipeline_options=True,
job_name_variable_key="job_name")
Review comment:
What is job_name_variable_key ?
##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -96,7 +100,107 @@ def set_current_dataflow_job_id(job_id):
)
-class BeamRunPythonPipelineOperator(BaseOperator, BeamDataflowMixin):
+class BeamBasePipelineOperator(BaseOperator, BeamDataflowMixin, ABC):
+ """
+ Abstract base class for Beam Pipeline Operators.
+
+ :param runner: Runner on which pipeline will be run. By default
"DirectRunner" is being used.
+ Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+ See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+ See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+ :type runner: str
+ :param default_pipeline_options: Map of default pipeline options.
+ :type default_pipeline_options: dict
+ :param pipeline_options: Map of pipeline options.The key must be a
dictionary.
+ The value can contain different types:
+
+ * If the value is None, the single option - ``--key`` (without value)
will be added.
+ * If the value is False, this option will be skipped
+ * If the value is True, the single option - ``--key`` (without value)
will be added.
+ * If the value is list, the many options will be added for each key.
+ If the value is ``['A', 'B']`` and the key is ``key`` then the
``--key=A --key-B`` options
+ will be left
+ * Other value types will be replaced with the Python textual
representation.
+
+ When defining labels (``labels`` option), you can also provide a
dictionary.
+ :type pipeline_options: dict
+ :param gcp_conn_id: Optional.
+ The connection ID to use connecting to Google Cloud Storage if python
file is on GCS.
+ :type gcp_conn_id: str
+ :param delegate_to: Optional.
+ The account to impersonate using domain-wide delegation of authority,
+ if any. For this to work, the service account making the request must
have
+ domain-wide delegation enabled.
+ :type delegate_to: str
+ :param dataflow_config: Dataflow configuration, used when runner type is
set to DataflowRunner,
+ (optional) defaults to None.
+ :type dataflow_config: DataflowConfiguration
+ """
+
+ def __init__(
+ self,
+ *,
+ runner: str = "DirectRunner",
+ default_pipeline_options: Optional[dict] = None,
+ pipeline_options: Optional[dict] = None,
+ gcp_conn_id: str = "google_cloud_default",
+ delegate_to: Optional[str] = None,
+ dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+ **kwargs,
+ ) -> None:
+ super().__init__(**kwargs)
+ self.runner = runner
+ self.default_pipeline_options = default_pipeline_options or {}
+ self.pipeline_options = pipeline_options or {}
+ self.pipeline_options.setdefault("labels", {}).update(
+ {"airflow-version": "v" + version.replace(".", "-").replace("+",
"-")}
+ )
+ self.gcp_conn_id = gcp_conn_id
+ self.delegate_to = delegate_to
+ if isinstance(dataflow_config, dict):
+ self.dataflow_config = DataflowConfiguration(**dataflow_config)
+ else:
+ self.dataflow_config = dataflow_config or DataflowConfiguration()
+ self.beam_hook: Optional[BeamHook] = None
+ self.dataflow_hook: Optional[DataflowHook] = None
+ self.dataflow_job_id: Optional[str] = None
+
+ if self.dataflow_config and self.runner.lower() !=
BeamRunnerType.DataflowRunner.lower():
+ self.log.warning(
+ "dataflow_config is defined but runner is different than
DataflowRunner (%s)", self.runner
+ )
+
+ def _init_pipeline_options(
+ self,
+ format_pipeline_options: bool = False,
+ job_name_variable_key: Optional[str] = None,
+ ) -> Tuple[bool, Optional[str], dict, Optional[Callable[[str], None]]]:
+ self.beam_hook = BeamHook(runner=self.runner)
+ pipeline_options = self.default_pipeline_options.copy()
+ process_line_callback: Optional[Callable[[str], None]] = None
+ is_dataflow = self.runner.lower() ==
BeamRunnerType.DataflowRunner.lower()
+ dataflow_job_name: Optional[str] = None
+
+ if is_dataflow:
+ dataflow_job_name, pipeline_options, process_line_callback =
self._set_dataflow(
+ pipeline_options=pipeline_options,
+ job_name_variable_key=job_name_variable_key,
+ )
+
+ pipeline_options.update(self.pipeline_options)
+
+ formatted_pipeline_options = pipeline_options
Review comment:
Perhaps rename this to something more explicit like format_as_snake
case, or something like that.
##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -470,3 +529,144 @@ def on_kill(self) -> None:
job_id=self.dataflow_job_id,
project_id=self.dataflow_config.project_id,
)
+
+
+class BeamRunGoPipelineOperator(BeamBasePipelineOperator):
+ """
+ Launching Apache Beam pipelines written in Go. Note that both
+ ``default_pipeline_options`` and ``pipeline_options`` will be merged to
specify pipeline
+ execution parameter, and ``default_pipeline_options`` is expected to save
+ high-level options, for instances, project and zone information, which
+ apply to all beam operators in the DAG.
+
+ .. seealso::
+ For more information on how to use this operator, take a look at the
guide:
+ :ref:`howto/operator:BeamRunGoPipelineOperator`
+
+ .. seealso::
+ For more detail on Apache Beam have a look at the reference:
+ https://beam.apache.org/documentation/
+
+ :param go_file: Reference to the Go Apache Beam pipeline e.g.,
+ /some/local/file/path/to/your/go/pipeline/file.go
+ :type go_file: str
+ :param runner: Runner on which pipeline will be run. By default
"DirectRunner" is being used.
+ Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+ See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+ See: https://beam.apache.org/documentation/runners/capability-matrix/
+ :type runner: str
+ :param default_pipeline_options: (optional) Map of default pipeline
options.
+ :type default_pipeline_options: dict
+ :param pipeline_options: (optional) Map of pipeline options.The key must
be a dictionary.
+ The value can contain different types:
+
+ * If the value is None, the single option - ``--key`` (without value)
will be added.
+ * If the value is False, this option will be skipped
+ * If the value is True, the single option - ``--key`` (without value)
will be added.
+ * If the value is list, the many options will be added for each key.
+ If the value is ``['A', 'B']`` and the key is ``key`` then the
``--key=A --key-B`` options
+ will be left
+ * Other value types will be replaced with the Python textual
representation.
+
+ When defining labels (``labels`` option), you can also provide a
dictionary.
+ :type pipeline_options: dict
+ :param gcp_conn_id: (optional)
+ The connection ID to use connecting to Google Cloud Storage if python
file is on GCS.
+ :type gcp_conn_id: str
+ :param delegate_to: (optional)
+ The account to impersonate using domain-wide delegation of authority,
+ if any. For this to work, the service account making the request must
have
+ domain-wide delegation enabled.
+ :type delegate_to: str
+ :param dataflow_config: (optional) Dataflow configuration, used when
runner type is set to DataflowRunner
+ :type dataflow_config: Union[dict,
providers.google.cloud.operators.dataflow.DataflowConfiguration]
+ """
+
+ template_fields = [
+ "go_file",
+ "runner",
+ "pipeline_options",
+ "default_pipeline_options",
+ "dataflow_config",
+ ]
+ template_fields_renderers = {'dataflow_config': 'json',
'pipeline_options': 'json'}
+
+ def __init__(
+ self,
+ *,
+ go_file: str,
+ runner: str = "DirectRunner",
+ default_pipeline_options: Optional[dict] = None,
+ pipeline_options: Optional[dict] = None,
+ gcp_conn_id: str = "google_cloud_default",
+ delegate_to: Optional[str] = None,
+ dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+ **kwargs,
+ ) -> None:
+ super().__init__(
+ runner=runner,
+ default_pipeline_options=default_pipeline_options,
+ pipeline_options=pipeline_options,
+ gcp_conn_id=gcp_conn_id,
+ delegate_to=delegate_to,
+ dataflow_config=dataflow_config,
+ **kwargs,
+ )
+
+ self.go_file = go_file
+ self.should_init_go_module = False
+
+ def execute(self, context):
+ """Execute the Apache Beam Pipeline."""
+ (
+ is_dataflow,
+ dataflow_job_name,
+ formatted_pipeline_options,
+ process_line_callback,
+ ) = self._init_pipeline_options(format_pipeline_options=True,
job_name_variable_key="job_name")
Review comment:
should go also be formatting the options like python?
##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -96,7 +100,107 @@ def set_current_dataflow_job_id(job_id):
)
-class BeamRunPythonPipelineOperator(BaseOperator, BeamDataflowMixin):
+class BeamBasePipelineOperator(BaseOperator, BeamDataflowMixin, ABC):
+ """
+ Abstract base class for Beam Pipeline Operators.
+
+ :param runner: Runner on which pipeline will be run. By default
"DirectRunner" is being used.
+ Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+ See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+ See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+ :type runner: str
+ :param default_pipeline_options: Map of default pipeline options.
+ :type default_pipeline_options: dict
+ :param pipeline_options: Map of pipeline options.The key must be a
dictionary.
+ The value can contain different types:
+
+ * If the value is None, the single option - ``--key`` (without value)
will be added.
+ * If the value is False, this option will be skipped
+ * If the value is True, the single option - ``--key`` (without value)
will be added.
+ * If the value is list, the many options will be added for each key.
+ If the value is ``['A', 'B']`` and the key is ``key`` then the
``--key=A --key-B`` options
+ will be left
+ * Other value types will be replaced with the Python textual
representation.
+
+ When defining labels (``labels`` option), you can also provide a
dictionary.
Review comment:
why is labels special in this case?
##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -470,3 +529,144 @@ def on_kill(self) -> None:
job_id=self.dataflow_job_id,
project_id=self.dataflow_config.project_id,
)
+
+
+class BeamRunGoPipelineOperator(BeamBasePipelineOperator):
+ """
+ Launching Apache Beam pipelines written in Go. Note that both
+ ``default_pipeline_options`` and ``pipeline_options`` will be merged to
specify pipeline
+ execution parameter, and ``default_pipeline_options`` is expected to save
+ high-level options, for instances, project and zone information, which
+ apply to all beam operators in the DAG.
+
+ .. seealso::
+ For more information on how to use this operator, take a look at the
guide:
+ :ref:`howto/operator:BeamRunGoPipelineOperator`
+
+ .. seealso::
+ For more detail on Apache Beam have a look at the reference:
+ https://beam.apache.org/documentation/
+
+ :param go_file: Reference to the Go Apache Beam pipeline e.g.,
+ /some/local/file/path/to/your/go/pipeline/file.go
+ :type go_file: str
+ :param runner: Runner on which pipeline will be run. By default
"DirectRunner" is being used.
+ Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+ See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+ See: https://beam.apache.org/documentation/runners/capability-matrix/
+ :type runner: str
+ :param default_pipeline_options: (optional) Map of default pipeline
options.
+ :type default_pipeline_options: dict
+ :param pipeline_options: (optional) Map of pipeline options.The key must
be a dictionary.
Review comment:
Is it possible to avoid these repated doc strings and refer to the base
class?
##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -470,3 +529,144 @@ def on_kill(self) -> None:
job_id=self.dataflow_job_id,
project_id=self.dataflow_config.project_id,
)
+
+
+class BeamRunGoPipelineOperator(BeamBasePipelineOperator):
+ """
+ Launching Apache Beam pipelines written in Go. Note that both
+ ``default_pipeline_options`` and ``pipeline_options`` will be merged to
specify pipeline
+ execution parameter, and ``default_pipeline_options`` is expected to save
+ high-level options, for instances, project and zone information, which
+ apply to all beam operators in the DAG.
+
+ .. seealso::
+ For more information on how to use this operator, take a look at the
guide:
+ :ref:`howto/operator:BeamRunGoPipelineOperator`
+
+ .. seealso::
+ For more detail on Apache Beam have a look at the reference:
+ https://beam.apache.org/documentation/
+
+ :param go_file: Reference to the Go Apache Beam pipeline e.g.,
+ /some/local/file/path/to/your/go/pipeline/file.go
+ :type go_file: str
+ :param runner: Runner on which pipeline will be run. By default
"DirectRunner" is being used.
+ Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+ See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+ See: https://beam.apache.org/documentation/runners/capability-matrix/
+ :type runner: str
+ :param default_pipeline_options: (optional) Map of default pipeline
options.
+ :type default_pipeline_options: dict
+ :param pipeline_options: (optional) Map of pipeline options.The key must
be a dictionary.
+ The value can contain different types:
+
+ * If the value is None, the single option - ``--key`` (without value)
will be added.
+ * If the value is False, this option will be skipped
+ * If the value is True, the single option - ``--key`` (without value)
will be added.
+ * If the value is list, the many options will be added for each key.
+ If the value is ``['A', 'B']`` and the key is ``key`` then the
``--key=A --key-B`` options
+ will be left
+ * Other value types will be replaced with the Python textual
representation.
+
+ When defining labels (``labels`` option), you can also provide a
dictionary.
+ :type pipeline_options: dict
+ :param gcp_conn_id: (optional)
+ The connection ID to use connecting to Google Cloud Storage if python
file is on GCS.
+ :type gcp_conn_id: str
+ :param delegate_to: (optional)
+ The account to impersonate using domain-wide delegation of authority,
+ if any. For this to work, the service account making the request must
have
+ domain-wide delegation enabled.
+ :type delegate_to: str
+ :param dataflow_config: (optional) Dataflow configuration, used when
runner type is set to DataflowRunner
+ :type dataflow_config: Union[dict,
providers.google.cloud.operators.dataflow.DataflowConfiguration]
+ """
+
+ template_fields = [
+ "go_file",
+ "runner",
+ "pipeline_options",
+ "default_pipeline_options",
+ "dataflow_config",
+ ]
+ template_fields_renderers = {'dataflow_config': 'json',
'pipeline_options': 'json'}
+
+ def __init__(
+ self,
+ *,
+ go_file: str,
+ runner: str = "DirectRunner",
+ default_pipeline_options: Optional[dict] = None,
+ pipeline_options: Optional[dict] = None,
+ gcp_conn_id: str = "google_cloud_default",
+ delegate_to: Optional[str] = None,
+ dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+ **kwargs,
+ ) -> None:
+ super().__init__(
+ runner=runner,
+ default_pipeline_options=default_pipeline_options,
+ pipeline_options=pipeline_options,
+ gcp_conn_id=gcp_conn_id,
+ delegate_to=delegate_to,
+ dataflow_config=dataflow_config,
+ **kwargs,
+ )
+
+ self.go_file = go_file
+ self.should_init_go_module = False
+
+ def execute(self, context):
+ """Execute the Apache Beam Pipeline."""
+ (
+ is_dataflow,
+ dataflow_job_name,
+ formatted_pipeline_options,
+ process_line_callback,
+ ) = self._init_pipeline_options(format_pipeline_options=True,
job_name_variable_key="job_name")
+
+ with ExitStack() as exit_stack:
+ if self.go_file.lower().startswith("gs://"):
+ gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+ # Go doesn't allow `go mod tidy` inside root tmp dir, creating
a go subdir to put files inside
+ if not os.path.isdir("/tmp/go"):
+ os.mkdir("/tmp/go")
+ tmp_gcs_file = exit_stack.enter_context(
+ gcs_hook.provide_file(object_url=self.go_file,
dir="/tmp/go")
+ )
+ self.go_file = tmp_gcs_file.name
+ self.should_init_go_module = True
+
+ if is_dataflow:
+ with self.dataflow_hook.provide_authorized_gcloud():
+ self.beam_hook.start_go_pipeline(
+ variables=formatted_pipeline_options,
+ go_file=self.go_file,
+ process_line_callback=process_line_callback,
+ should_init_module=self.should_init_go_module,
+ )
+
+ self.dataflow_hook.wait_for_done(
+ job_name=dataflow_job_name,
+ location=self.dataflow_config.location,
+ job_id=self.dataflow_job_id,
+ multiple_jobs=False,
+ )
+
+ else:
+ self.beam_hook.start_go_pipeline(
+ variables=formatted_pipeline_options,
+ go_file=self.go_file,
+ process_line_callback=process_line_callback,
+ should_init_module=self.should_init_go_module,
+ )
+
+ return {"dataflow_job_id": self.dataflow_job_id}
Review comment:
should this be inside is dataflow case? what is dataflow job id
otherwise?
##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -470,3 +529,144 @@ def on_kill(self) -> None:
job_id=self.dataflow_job_id,
project_id=self.dataflow_config.project_id,
)
+
+
+class BeamRunGoPipelineOperator(BeamBasePipelineOperator):
+ """
+ Launching Apache Beam pipelines written in Go. Note that both
+ ``default_pipeline_options`` and ``pipeline_options`` will be merged to
specify pipeline
+ execution parameter, and ``default_pipeline_options`` is expected to save
+ high-level options, for instances, project and zone information, which
+ apply to all beam operators in the DAG.
+
+ .. seealso::
+ For more information on how to use this operator, take a look at the
guide:
+ :ref:`howto/operator:BeamRunGoPipelineOperator`
+
+ .. seealso::
+ For more detail on Apache Beam have a look at the reference:
+ https://beam.apache.org/documentation/
+
+ :param go_file: Reference to the Go Apache Beam pipeline e.g.,
+ /some/local/file/path/to/your/go/pipeline/file.go
+ :type go_file: str
+ :param runner: Runner on which pipeline will be run. By default
"DirectRunner" is being used.
+ Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+ See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+ See: https://beam.apache.org/documentation/runners/capability-matrix/
+ :type runner: str
+ :param default_pipeline_options: (optional) Map of default pipeline
options.
+ :type default_pipeline_options: dict
+ :param pipeline_options: (optional) Map of pipeline options.The key must
be a dictionary.
+ The value can contain different types:
+
+ * If the value is None, the single option - ``--key`` (without value)
will be added.
+ * If the value is False, this option will be skipped
+ * If the value is True, the single option - ``--key`` (without value)
will be added.
+ * If the value is list, the many options will be added for each key.
+ If the value is ``['A', 'B']`` and the key is ``key`` then the
``--key=A --key-B`` options
+ will be left
+ * Other value types will be replaced with the Python textual
representation.
+
+ When defining labels (``labels`` option), you can also provide a
dictionary.
+ :type pipeline_options: dict
+ :param gcp_conn_id: (optional)
+ The connection ID to use connecting to Google Cloud Storage if python
file is on GCS.
+ :type gcp_conn_id: str
+ :param delegate_to: (optional)
+ The account to impersonate using domain-wide delegation of authority,
+ if any. For this to work, the service account making the request must
have
+ domain-wide delegation enabled.
+ :type delegate_to: str
+ :param dataflow_config: (optional) Dataflow configuration, used when
runner type is set to DataflowRunner
+ :type dataflow_config: Union[dict,
providers.google.cloud.operators.dataflow.DataflowConfiguration]
+ """
+
+ template_fields = [
+ "go_file",
+ "runner",
+ "pipeline_options",
+ "default_pipeline_options",
+ "dataflow_config",
+ ]
+ template_fields_renderers = {'dataflow_config': 'json',
'pipeline_options': 'json'}
+
+ def __init__(
+ self,
+ *,
+ go_file: str,
+ runner: str = "DirectRunner",
+ default_pipeline_options: Optional[dict] = None,
+ pipeline_options: Optional[dict] = None,
+ gcp_conn_id: str = "google_cloud_default",
+ delegate_to: Optional[str] = None,
+ dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+ **kwargs,
+ ) -> None:
+ super().__init__(
+ runner=runner,
+ default_pipeline_options=default_pipeline_options,
+ pipeline_options=pipeline_options,
+ gcp_conn_id=gcp_conn_id,
+ delegate_to=delegate_to,
+ dataflow_config=dataflow_config,
+ **kwargs,
+ )
+
+ self.go_file = go_file
+ self.should_init_go_module = False
+
+ def execute(self, context):
+ """Execute the Apache Beam Pipeline."""
+ (
+ is_dataflow,
+ dataflow_job_name,
+ formatted_pipeline_options,
+ process_line_callback,
+ ) = self._init_pipeline_options(format_pipeline_options=True,
job_name_variable_key="job_name")
+
+ with ExitStack() as exit_stack:
+ if self.go_file.lower().startswith("gs://"):
+ gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+ # Go doesn't allow `go mod tidy` inside root tmp dir, creating
a go subdir to put files inside
+ if not os.path.isdir("/tmp/go"):
+ os.mkdir("/tmp/go")
+ tmp_gcs_file = exit_stack.enter_context(
+ gcs_hook.provide_file(object_url=self.go_file,
dir="/tmp/go")
+ )
+ self.go_file = tmp_gcs_file.name
+ self.should_init_go_module = True
+
+ if is_dataflow:
+ with self.dataflow_hook.provide_authorized_gcloud():
+ self.beam_hook.start_go_pipeline(
+ variables=formatted_pipeline_options,
+ go_file=self.go_file,
+ process_line_callback=process_line_callback,
+ should_init_module=self.should_init_go_module,
+ )
+
+ self.dataflow_hook.wait_for_done(
+ job_name=dataflow_job_name,
+ location=self.dataflow_config.location,
+ job_id=self.dataflow_job_id,
+ multiple_jobs=False,
+ )
+
Review comment:
remove empty line?
--
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]