Skip to content

Commit

Permalink
Fix incorrect Usage of Optional[bool] (#11138)
Browse files Browse the repository at this point in the history
Optional[bool] = Union[None, bool]

There were incorrect usages where the default was already set to
a boolean value but still Optional was used
  • Loading branch information
kaxil authored Sep 24, 2020
1 parent af3c677 commit e3f96ce
Show file tree
Hide file tree
Showing 11 changed files with 30 additions and 30 deletions.
2 changes: 1 addition & 1 deletion airflow/models/dagrun.py
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ def find(
execution_date: Optional[datetime] = None,
state: Optional[str] = None,
external_trigger: Optional[bool] = None,
no_backfills: Optional[bool] = False,
no_backfills: bool = False,
run_type: Optional[DagRunType] = None,
session: Session = None,
execution_start_date: Optional[datetime] = None,
Expand Down
26 changes: 13 additions & 13 deletions airflow/models/taskinstance.py
Original file line number Diff line number Diff line change
Expand Up @@ -363,15 +363,15 @@ def command_as_list( # pylint: disable=too-many-arguments
def generate_command(dag_id: str, # pylint: disable=too-many-arguments
task_id: str,
execution_date: datetime,
mark_success: Optional[bool] = False,
ignore_all_deps: Optional[bool] = False,
ignore_depends_on_past: Optional[bool] = False,
ignore_task_deps: Optional[bool] = False,
ignore_ti_state: Optional[bool] = False,
local: Optional[bool] = False,
mark_success: bool = False,
ignore_all_deps: bool = False,
ignore_depends_on_past: bool = False,
ignore_task_deps: bool = False,
ignore_ti_state: bool = False,
local: bool = False,
pickle_id: Optional[int] = None,
file_path: Optional[str] = None,
raw: Optional[bool] = False,
raw: bool = False,
job_id: Optional[str] = None,
pool: Optional[str] = None,
cfg_path: Optional[str] = None
Expand All @@ -386,20 +386,20 @@ def generate_command(dag_id: str, # pylint: disable=too-many-arguments
:param execution_date: Execution date for the task
:type execution_date: datetime
:param mark_success: Whether to mark the task as successful
:type mark_success: Optional[bool]
:type mark_success: bool
:param ignore_all_deps: Ignore all ignorable dependencies.
Overrides the other ignore_* parameters.
:type ignore_all_deps: Optional[bool]
:type ignore_all_deps: bool
:param ignore_depends_on_past: Ignore depends_on_past parameter of DAGs
(e.g. for Backfills)
:type ignore_depends_on_past: Optional[bool]
:type ignore_depends_on_past: bool
:param ignore_task_deps: Ignore task-specific dependencies such as depends_on_past
and trigger rule
:type ignore_task_deps: Optional[bool]
:type ignore_task_deps: bool
:param ignore_ti_state: Ignore the task instance's previous failure/success
:type ignore_ti_state: Optional[bool]
:type ignore_ti_state: bool
:param local: Whether to run the task locally
:type local: Optional[bool]
:type local: bool
:param pickle_id: If the DAG was serialized to the DB, the ID
associated with the pickled DAG
:type pickle_id: Optional[int]
Expand Down
2 changes: 1 addition & 1 deletion airflow/operators/sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -273,7 +273,7 @@ def __init__(
date_filter_column: Optional[str] = "ds",
days_back: SupportsAbs[int] = -7,
ratio_formula: Optional[str] = "max_over_min",
ignore_zero: Optional[bool] = True,
ignore_zero: bool = True,
conn_id: Optional[str] = None,
**kwargs,
):
Expand Down
2 changes: 1 addition & 1 deletion airflow/providers/amazon/aws/operators/s3_bucket.py
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ class S3DeleteBucketOperator(BaseOperator):
def __init__(
self,
bucket_name,
force_delete: Optional[bool] = False,
force_delete: bool = False,
aws_conn_id: Optional[str] = "aws_default",
**kwargs,
) -> None:
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/transfers/mysql_to_s3.py
Original file line number Diff line number Diff line change
Expand Up @@ -80,8 +80,8 @@ def __init__(
aws_conn_id: str = 'aws_default',
verify: Optional[Union[bool, str]] = None,
pd_csv_kwargs: Optional[dict] = None,
index: Optional[bool] = False,
header: Optional[bool] = False,
index: bool = False,
header: bool = False,
**kwargs,
) -> None:
super().__init__(**kwargs)
Expand Down
2 changes: 1 addition & 1 deletion airflow/providers/apache/hive/hooks/hive.py
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ def run_cli(
self,
hql: Union[str, Text],
schema: Optional[str] = None,
verbose: Optional[bool] = True,
verbose: bool = True,
hive_conf: Optional[Dict[Any, Any]] = None,
) -> Any:
"""
Expand Down
2 changes: 1 addition & 1 deletion airflow/providers/apache/kylin/operators/kylin_cube.py
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ def __init__(
offset_start: Optional[str] = None,
offset_end: Optional[str] = None,
segment_name: Optional[str] = None,
is_track_job: Optional[bool] = False,
is_track_job: bool = False,
interval: int = 60,
timeout: int = 60 * 60 * 24,
eager_error_status=("ERROR", "DISCARDED", "KILLED", "SUICIDAL", "STOPPED"),
Expand Down
12 changes: 6 additions & 6 deletions airflow/providers/apache/pinot/hooks/pinot.py
Original file line number Diff line number Diff line change
Expand Up @@ -73,14 +73,14 @@ def __init__(
def get_conn(self) -> Any:
return self.conn

def add_schema(self, schema_file: str, with_exec: Optional[bool] = True) -> Any:
def add_schema(self, schema_file: str, with_exec: bool = True) -> Any:
"""
Add Pinot schema by run AddSchema command
:param schema_file: Pinot schema file
:type schema_file: str
:param with_exec: bool
:type with_exec: Optional[bool]
:type with_exec: bool
"""
cmd = ["AddSchema"]
cmd += ["-controllerHost", self.host]
Expand All @@ -90,14 +90,14 @@ def add_schema(self, schema_file: str, with_exec: Optional[bool] = True) -> Any:
cmd += ["-exec"]
self.run_cli(cmd)

def add_table(self, file_path: str, with_exec: Optional[bool] = True) -> Any:
def add_table(self, file_path: str, with_exec: bool = True) -> Any:
"""
Add Pinot table with run AddTable command
:param file_path: Pinot table configure file
:type file_path: str
:param with_exec: bool
:type with_exec: Optional[bool]
:type with_exec: bool
"""
cmd = ["AddTable"]
cmd += ["-controllerHost", self.host]
Expand Down Expand Up @@ -206,14 +206,14 @@ def upload_segment(self, segment_dir: str, table_name: Optional[str] = None) ->
cmd += ["-tableName", table_name]
self.run_cli(cmd)

def run_cli(self, cmd: List[str], verbose: Optional[bool] = True) -> str:
def run_cli(self, cmd: List[str], verbose: bool = True) -> str:
"""
Run command with pinot-admin.sh
:param cmd: List of command going to be run by pinot-admin.sh script
:type cmd: list
:param verbose:
:type verbose: Optional[bool]
:type verbose: bool
"""
command = [self.cmd_path]
command.extend(cmd)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ def __init__(
self,
*,
application_name: str,
attach_log: Optional[bool] = False,
attach_log: bool = False,
namespace: Optional[str] = None,
kubernetes_conn_id: str = "kubernetes_default",
**kwargs,
Expand Down
2 changes: 1 addition & 1 deletion airflow/providers/docker/operators/docker.py
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,7 @@ def __init__(
dns_search: Optional[List[str]] = None,
auto_remove: bool = False,
shm_size: Optional[int] = None,
tty: Optional[bool] = False,
tty: bool = False,
cap_add: Optional[Iterable[str]] = None,
extra_hosts: Optional[Dict[str, str]] = None,
**kwargs,
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/google/cloud/operators/bigquery.py
Original file line number Diff line number Diff line change
Expand Up @@ -621,13 +621,13 @@ def __init__(
sql: Union[str, Iterable],
destination_dataset_table: Optional[str] = None,
write_disposition: Optional[str] = 'WRITE_EMPTY',
allow_large_results: Optional[bool] = False,
allow_large_results: bool = False,
flatten_results: Optional[bool] = None,
gcp_conn_id: Optional[str] = 'google_cloud_default',
bigquery_conn_id: Optional[str] = None,
delegate_to: Optional[str] = None,
udf_config: Optional[list] = None,
use_legacy_sql: Optional[bool] = True,
use_legacy_sql: bool = True,
maximum_billing_tier: Optional[int] = None,
maximum_bytes_billed: Optional[float] = None,
create_disposition: Optional[str] = 'CREATE_IF_NEEDED',
Expand Down

0 comments on commit e3f96ce

Please sign in to comment.