Skip to content

Commit

Permalink
Fix MyPy Errors for Apache Beam (and Dataflow) provider. (#20301)
Browse files Browse the repository at this point in the history
  • Loading branch information
kazanzhy authored Dec 15, 2021
1 parent 5712e2b commit 43efde6
Show file tree
Hide file tree
Showing 3 changed files with 19 additions and 12 deletions.
26 changes: 17 additions & 9 deletions airflow/providers/apache/beam/operators/beam.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,9 @@ class BeamDataflowMixin(metaclass=ABCMeta):
"""

dataflow_hook: Optional[DataflowHook]
dataflow_config: Optional[DataflowConfiguration]
dataflow_config: DataflowConfiguration
gcp_conn_id: str
delegate_to: Optional[str]

def _set_dataflow(
self, pipeline_options: dict, job_name_variable_key: Optional[str] = None
Expand Down Expand Up @@ -198,11 +200,17 @@ def __init__(
self.py_system_site_packages = py_system_site_packages
self.gcp_conn_id = gcp_conn_id
self.delegate_to = delegate_to
self.dataflow_config = dataflow_config or {}
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
Expand All @@ -216,9 +224,6 @@ def execute(self, context):
is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
dataflow_job_name: Optional[str] = None

if isinstance(self.dataflow_config, dict):
self.dataflow_config = DataflowConfiguration(**self.dataflow_config)

if is_dataflow:
dataflow_job_name, pipeline_options, process_line_callback = self._set_dataflow(
pipeline_options=pipeline_options, job_name_variable_key="job_name"
Expand Down Expand Up @@ -366,14 +371,20 @@ def __init__(
self.default_pipeline_options = default_pipeline_options or {}
self.pipeline_options = pipeline_options or {}
self.job_class = job_class
self.dataflow_config = dataflow_config or {}
self.gcp_conn_id = gcp_conn_id
self.delegate_to = delegate_to
self.dataflow_job_id = None
self.dataflow_hook: Optional[DataflowHook] = None
self.beam_hook: Optional[BeamHook] = None
self._dataflow_job_name: 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
Expand All @@ -387,9 +398,6 @@ def execute(self, context):
is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
dataflow_job_name: Optional[str] = None

if isinstance(self.dataflow_config, dict):
self.dataflow_config = DataflowConfiguration(**self.dataflow_config)

if is_dataflow:
dataflow_job_name, pipeline_options, process_line_callback = self._set_dataflow(
pipeline_options=pipeline_options, job_name_variable_key=None
Expand Down
3 changes: 1 addition & 2 deletions airflow/providers/google/cloud/hooks/dataflow.py
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,6 @@ def __init__(
self._jobs: Optional[List[dict]] = None
self.drain_pipeline = drain_pipeline
self._wait_until_finished = wait_until_finished
self._jobs: Optional[List[dict]] = None

def is_job_running(self) -> bool:
"""
Expand Down Expand Up @@ -1064,7 +1063,7 @@ def start_sql_job(
DeprecationWarning,
stacklevel=3,
)
on_new_job_id_callback(job.get("id"))
on_new_job_id_callback(job["id"])

if on_new_job_callback:
on_new_job_callback(job)
Expand Down
2 changes: 1 addition & 1 deletion airflow/providers/google/cloud/operators/dataflow.py
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ class DataflowConfiguration:
def __init__(
self,
*,
job_name: Optional[str] = "{{task.task_id}}",
job_name: str = "{{task.task_id}}",
append_job_name: bool = True,
project_id: Optional[str] = None,
location: Optional[str] = DEFAULT_DATAFLOW_LOCATION,
Expand Down

0 comments on commit 43efde6

Please sign in to comment.
  翻译: