[AIRFLOW-2800] Remove low-hanging linting errors
This commit is contained in:
Родитель
6fdc79980b
Коммит
06584fc4b1
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
@ -80,11 +80,12 @@ class AirflowMacroPlugin(object):
|
|||
def __init__(self, namespace):
|
||||
self.namespace = namespace
|
||||
|
||||
from airflow import operators
|
||||
|
||||
from airflow import operators # noqa: E402
|
||||
from airflow import sensors # noqa: E402
|
||||
from airflow import hooks
|
||||
from airflow import executors
|
||||
from airflow import macros
|
||||
from airflow import hooks # noqa: E402
|
||||
from airflow import executors # noqa: E402
|
||||
from airflow import macros # noqa: E402
|
||||
|
||||
operators._integrate_plugins()
|
||||
sensors._integrate_plugins() # noqa: E402
|
||||
|
|
|
@ -62,7 +62,7 @@ def get_ldap_connection(dn=None, password=None):
|
|||
cacert = configuration.conf.get("ldap", "cacert")
|
||||
tls_configuration = Tls(validate=ssl.CERT_REQUIRED, ca_certs_file=cacert)
|
||||
use_ssl = True
|
||||
except:
|
||||
except Exception:
|
||||
pass
|
||||
|
||||
server = Server(configuration.conf.get("ldap", "uri"), use_ssl, tls_configuration)
|
||||
|
@ -94,7 +94,7 @@ def groups_user(conn, search_base, user_filter, user_name_att, username):
|
|||
search_filter = "(&({0})({1}={2}))".format(user_filter, user_name_att, username)
|
||||
try:
|
||||
memberof_attr = configuration.conf.get("ldap", "group_member_attr")
|
||||
except:
|
||||
except Exception:
|
||||
memberof_attr = "memberOf"
|
||||
res = conn.search(native(search_base), native(search_filter),
|
||||
attributes=[native(memberof_attr)])
|
||||
|
|
|
@ -72,7 +72,7 @@ def _parse_s3_config(config_file_name, config_format='boto', profile=None):
|
|||
try:
|
||||
access_key = config.get(cred_section, key_id_option)
|
||||
secret_key = config.get(cred_section, secret_key_option)
|
||||
except:
|
||||
except Exception:
|
||||
logging.warning("Option Error in parsing s3 config file")
|
||||
raise
|
||||
return access_key, secret_key
|
||||
|
|
|
@ -139,7 +139,7 @@ class AWSBatchOperator(BaseOperator):
|
|||
if response['jobs'][-1]['status'] in ['SUCCEEDED', 'FAILED']:
|
||||
retry = False
|
||||
|
||||
sleep( 1 + pow(retries * 0.1, 2))
|
||||
sleep(1 + pow(retries * 0.1, 2))
|
||||
retries += 1
|
||||
|
||||
def _check_success_task(self):
|
||||
|
|
|
@ -112,14 +112,14 @@ class JsonCoder(object):
|
|||
@beam.ptransform_fn
|
||||
def MakeSummary(pcoll, metric_fn, metric_keys): # pylint: disable=invalid-name
|
||||
return (
|
||||
pcoll
|
||||
| "ApplyMetricFnPerInstance" >> beam.Map(metric_fn)
|
||||
| "PairWith1" >> beam.Map(lambda tup: tup + (1,))
|
||||
| "SumTuple" >> beam.CombineGlobally(beam.combiners.TupleCombineFn(
|
||||
*([sum] * (len(metric_keys) + 1))))
|
||||
| "AverageAndMakeDict" >> beam.Map(
|
||||
pcoll |
|
||||
"ApplyMetricFnPerInstance" >> beam.Map(metric_fn) |
|
||||
"PairWith1" >> beam.Map(lambda tup: tup + (1,)) |
|
||||
"SumTuple" >> beam.CombineGlobally(beam.combiners.TupleCombineFn(
|
||||
*([sum] * (len(metric_keys) + 1)))) |
|
||||
"AverageAndMakeDict" >> beam.Map(
|
||||
lambda tup: dict(
|
||||
[(name, tup[i]/tup[-1]) for i, name in enumerate(metric_keys)] +
|
||||
[(name, tup[i] / tup[-1]) for i, name in enumerate(metric_keys)] +
|
||||
[("count", tup[-1])])))
|
||||
|
||||
|
||||
|
|
|
@ -173,7 +173,8 @@ class BaseJob(Base, LoggingMixin):
|
|||
if job.latest_heartbeat:
|
||||
sleep_for = max(
|
||||
0,
|
||||
self.heartrate - (timezone.utcnow() - job.latest_heartbeat).total_seconds())
|
||||
self.heartrate - (
|
||||
timezone.utcnow() - job.latest_heartbeat).total_seconds())
|
||||
|
||||
sleep(sleep_for)
|
||||
|
||||
|
@ -391,7 +392,7 @@ class DagFileProcessor(AbstractDagFileProcessor, LoggingMixin):
|
|||
log.info(
|
||||
"Processing %s took %.3f seconds", file_path, end_time - start_time
|
||||
)
|
||||
except:
|
||||
except Exception:
|
||||
# Log exceptions through the logging framework.
|
||||
log.exception("Got an exception! Propagating...")
|
||||
raise
|
||||
|
@ -1089,17 +1090,18 @@ class SchedulerJob(BaseJob):
|
|||
session
|
||||
.query(TI)
|
||||
.filter(TI.dag_id.in_(simple_dag_bag.dag_ids))
|
||||
.outerjoin(DR,
|
||||
and_(DR.dag_id == TI.dag_id,
|
||||
DR.execution_date == TI.execution_date))
|
||||
.filter(or_(DR.run_id == None,
|
||||
.outerjoin(
|
||||
DR,
|
||||
and_(DR.dag_id == TI.dag_id, DR.execution_date == TI.execution_date)
|
||||
)
|
||||
.filter(or_(DR.run_id == None, # noqa E711
|
||||
not_(DR.run_id.like(BackfillJob.ID_PREFIX + '%'))))
|
||||
.outerjoin(DM, DM.dag_id==TI.dag_id)
|
||||
.filter(or_(DM.dag_id == None,
|
||||
.outerjoin(DM, DM.dag_id == TI.dag_id)
|
||||
.filter(or_(DM.dag_id == None, # noqa E711
|
||||
not_(DM.is_paused)))
|
||||
)
|
||||
if None in states:
|
||||
ti_query = ti_query.filter(or_(TI.state == None, TI.state.in_(states)))
|
||||
ti_query = ti_query.filter(or_(TI.state == None, TI.state.in_(states))) # noqa E711
|
||||
else:
|
||||
ti_query = ti_query.filter(TI.state.in_(states))
|
||||
|
||||
|
@ -1121,7 +1123,8 @@ class SchedulerJob(BaseJob):
|
|||
for task_instance in task_instances_to_examine:
|
||||
pool_to_task_instances[task_instance.pool].append(task_instance)
|
||||
|
||||
task_concurrency_map = self.__get_task_concurrency_map(states=states_to_count_as_running, session=session)
|
||||
task_concurrency_map = self.__get_task_concurrency_map(
|
||||
states=states_to_count_as_running, session=session)
|
||||
|
||||
# Go through each pool, and queue up a task for execution if there are
|
||||
# any open slots in the pool.
|
||||
|
@ -1192,9 +1195,14 @@ class SchedulerJob(BaseJob):
|
|||
)
|
||||
continue
|
||||
|
||||
task_concurrency = simple_dag.get_task_special_arg(task_instance.task_id, 'task_concurrency')
|
||||
task_concurrency = simple_dag.get_task_special_arg(
|
||||
task_instance.task_id,
|
||||
'task_concurrency')
|
||||
if task_concurrency is not None:
|
||||
num_running = task_concurrency_map[((task_instance.dag_id, task_instance.task_id))]
|
||||
num_running = task_concurrency_map[
|
||||
((task_instance.dag_id, task_instance.task_id))
|
||||
]
|
||||
|
||||
if num_running >= task_concurrency:
|
||||
self.log.info("Not executing %s since the task concurrency for"
|
||||
" this task has been reached.", task_instance)
|
||||
|
@ -1214,7 +1222,8 @@ class SchedulerJob(BaseJob):
|
|||
|
||||
task_instance_str = "\n\t".join(
|
||||
["{}".format(x) for x in executable_tis])
|
||||
self.log.info("Setting the follow tasks to queued state:\n\t%s", task_instance_str)
|
||||
self.log.info(
|
||||
"Setting the follow tasks to queued state:\n\t%s", task_instance_str)
|
||||
# so these dont expire on commit
|
||||
for ti in executable_tis:
|
||||
copy_dag_id = ti.dag_id
|
||||
|
@ -1256,7 +1265,9 @@ class SchedulerJob(BaseJob):
|
|||
.filter(or_(*filter_for_ti_state_change)))
|
||||
|
||||
if None in acceptable_states:
|
||||
ti_query = ti_query.filter(or_(TI.state == None, TI.state.in_(acceptable_states)))
|
||||
ti_query = ti_query.filter(
|
||||
or_(TI.state == None, TI.state.in_(acceptable_states)) # noqa E711
|
||||
)
|
||||
else:
|
||||
ti_query = ti_query.filter(TI.state.in_(acceptable_states))
|
||||
|
||||
|
@ -1601,7 +1612,8 @@ class SchedulerJob(BaseJob):
|
|||
child.terminate()
|
||||
# TODO: Remove magic number
|
||||
timeout = 5
|
||||
self.log.info("Waiting up to %s seconds for processes to exit...", timeout)
|
||||
self.log.info(
|
||||
"Waiting up to %s seconds for processes to exit...", timeout)
|
||||
try:
|
||||
psutil.wait_procs(
|
||||
child_processes, timeout=timeout,
|
||||
|
@ -1658,7 +1670,9 @@ class SchedulerJob(BaseJob):
|
|||
self.log.info("Searching for files in %s", self.subdir)
|
||||
known_file_paths = list_py_file_paths(self.subdir)
|
||||
last_dag_dir_refresh_time = timezone.utcnow()
|
||||
self.log.info("There are %s files in %s", len(known_file_paths), self.subdir)
|
||||
self.log.info(
|
||||
"There are %s files in %s", len(known_file_paths), self.subdir)
|
||||
|
||||
processor_manager.set_file_paths(known_file_paths)
|
||||
|
||||
self.log.debug("Removing old import errors")
|
||||
|
@ -1671,7 +1685,9 @@ class SchedulerJob(BaseJob):
|
|||
if self.using_sqlite:
|
||||
# For the sqlite case w/ 1 thread, wait until the processor
|
||||
# is finished to avoid concurrent access to the DB.
|
||||
self.log.debug("Waiting for processors to finish since we're using sqlite")
|
||||
self.log.debug(
|
||||
"Waiting for processors to finish since we're using sqlite")
|
||||
|
||||
processor_manager.wait_until_finished()
|
||||
|
||||
# Send tasks for execution if available
|
||||
|
|
|
@ -7,13 +7,12 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
@ -39,4 +39,3 @@ def upgrade():
|
|||
|
||||
def downgrade():
|
||||
op.drop_index('dag_id_state', table_name='dag_run')
|
||||
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
@ -41,4 +41,3 @@ def upgrade():
|
|||
|
||||
def downgrade():
|
||||
op.rename_table('users', 'user')
|
||||
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,13 +7,12 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
@ -36,7 +36,7 @@ import sqlalchemy as sa
|
|||
|
||||
|
||||
def upgrade():
|
||||
# There can be data truncation here as LargeBinary can be smaller than the pickle
|
||||
# There can be data truncation here as LargeBinary can be smaller than the pickle
|
||||
# type.
|
||||
|
||||
# use batch_alter_table to support SQLite workaround
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
@ -34,6 +34,7 @@ from airflow.utils.log.logging_mixin import LoggingMixin
|
|||
|
||||
log = LoggingMixin().log
|
||||
|
||||
|
||||
class AirflowPluginException(Exception):
|
||||
pass
|
||||
|
||||
|
@ -107,6 +108,7 @@ def make_module(name, objects):
|
|||
module.__dict__.update((o.__name__, o) for o in objects)
|
||||
return module
|
||||
|
||||
|
||||
# Plugin components to integrate as modules
|
||||
operators_modules = []
|
||||
sensors_modules = []
|
||||
|
|
|
@ -47,7 +47,7 @@ try:
|
|||
TIMEZONE = pendulum.local_timezone()
|
||||
else:
|
||||
TIMEZONE = pendulum.timezone(tz)
|
||||
except:
|
||||
except Exception:
|
||||
pass
|
||||
log.info("Configured default timezone %s" % TIMEZONE)
|
||||
|
||||
|
@ -226,9 +226,9 @@ def configure_action_logging():
|
|||
|
||||
|
||||
try:
|
||||
from airflow_local_settings import *
|
||||
from airflow_local_settings import * # noqa F403 F401
|
||||
log.info("Loaded airflow_local_settings.")
|
||||
except:
|
||||
except Exception:
|
||||
pass
|
||||
|
||||
configure_logging()
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -160,7 +160,7 @@ def round_time(dt, delta, start_date=timezone.make_aware(datetime.min)):
|
|||
# We first search an upper limit for i for which start_date + upper * delta
|
||||
# exceeds dt.
|
||||
upper = 1
|
||||
while start_date + upper*delta < dt:
|
||||
while start_date + upper * delta < dt:
|
||||
# To speed up finding an upper limit we grow this exponentially by a
|
||||
# factor of 2
|
||||
upper *= 2
|
||||
|
@ -177,20 +177,20 @@ def round_time(dt, delta, start_date=timezone.make_aware(datetime.min)):
|
|||
# Invariant: start + lower * delta < dt <= start + upper * delta
|
||||
# If start_date + (lower + 1)*delta exceeds dt, then either lower or
|
||||
# lower+1 has to be the solution we are searching for
|
||||
if start_date + (lower + 1)*delta >= dt:
|
||||
if start_date + (lower + 1) * delta >= dt:
|
||||
# Check if start_date + (lower + 1)*delta or
|
||||
# start_date + lower*delta is closer to dt and return the solution
|
||||
if (
|
||||
(start_date + (lower + 1) * delta) - dt <=
|
||||
dt - (start_date + lower * delta)):
|
||||
return start_date + (lower + 1)*delta
|
||||
return start_date + (lower + 1) * delta
|
||||
else:
|
||||
return start_date + lower * delta
|
||||
|
||||
# We intersect the interval and either replace the lower or upper
|
||||
# limit with the candidate
|
||||
candidate = lower + (upper - lower) // 2
|
||||
if start_date + candidate*delta >= dt:
|
||||
if start_date + candidate * delta >= dt:
|
||||
upper = candidate
|
||||
else:
|
||||
lower = candidate
|
||||
|
@ -209,11 +209,11 @@ def infer_time_unit(time_seconds_arr):
|
|||
if len(time_seconds_arr) == 0:
|
||||
return 'hours'
|
||||
max_time_seconds = max(time_seconds_arr)
|
||||
if max_time_seconds <= 60*2:
|
||||
if max_time_seconds <= 60 * 2:
|
||||
return 'seconds'
|
||||
elif max_time_seconds <= 60*60*2:
|
||||
elif max_time_seconds <= 60 * 60 * 2:
|
||||
return 'minutes'
|
||||
elif max_time_seconds <= 24*60*60*2:
|
||||
elif max_time_seconds <= 24 * 60 * 60 * 2:
|
||||
return 'hours'
|
||||
else:
|
||||
return 'days'
|
||||
|
@ -224,11 +224,11 @@ def scale_time_units(time_seconds_arr, unit):
|
|||
Convert an array of time durations in seconds to the specified time unit.
|
||||
"""
|
||||
if unit == 'minutes':
|
||||
return list(map(lambda x: x*1.0/60, time_seconds_arr))
|
||||
return list(map(lambda x: x * 1.0 / 60, time_seconds_arr))
|
||||
elif unit == 'hours':
|
||||
return list(map(lambda x: x*1.0/(60*60), time_seconds_arr))
|
||||
return list(map(lambda x: x * 1.0 / (60 * 60), time_seconds_arr))
|
||||
elif unit == 'days':
|
||||
return list(map(lambda x: x*1.0/(24*60*60), time_seconds_arr))
|
||||
return list(map(lambda x: x * 1.0 / (24 * 60 * 60), time_seconds_arr))
|
||||
return time_seconds_arr
|
||||
|
||||
|
||||
|
|
|
@ -43,7 +43,7 @@ def create_session():
|
|||
yield session
|
||||
session.expunge_all()
|
||||
session.commit()
|
||||
except:
|
||||
except Exception:
|
||||
session.rollback()
|
||||
raise
|
||||
finally:
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -42,7 +42,7 @@ class S3TaskHandler(FileTaskHandler, LoggingMixin):
|
|||
try:
|
||||
from airflow.hooks.S3_hook import S3Hook
|
||||
return S3Hook(remote_conn_id)
|
||||
except:
|
||||
except Exception:
|
||||
self.log.error(
|
||||
'Could not create an S3Hook with connection id "%s". '
|
||||
'Please make sure that airflow[s3] is installed and '
|
||||
|
@ -139,7 +139,7 @@ class S3TaskHandler(FileTaskHandler, LoggingMixin):
|
|||
"""
|
||||
try:
|
||||
return self.hook.read_key(remote_log_location)
|
||||
except:
|
||||
except Exception:
|
||||
msg = 'Could not read logs from {}'.format(remote_log_location)
|
||||
self.log.exception(msg)
|
||||
# return error if needed
|
||||
|
@ -169,5 +169,5 @@ class S3TaskHandler(FileTaskHandler, LoggingMixin):
|
|||
replace=True,
|
||||
encrypt=configuration.conf.getboolean('core', 'ENCRYPT_S3_LOGS'),
|
||||
)
|
||||
except:
|
||||
except Exception:
|
||||
self.log.exception('Could not write logs to %s', remote_log_location)
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,13 +7,12 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -8,9 +8,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
@ -45,8 +45,11 @@ class GreaterEqualThan(EqualTo):
|
|||
|
||||
if field.data < other.data:
|
||||
d = {
|
||||
'other_label': hasattr(other, 'label') and other.label.text
|
||||
or self.fieldname,
|
||||
'other_label': (
|
||||
hasattr(other, 'label') and
|
||||
other.label.text or
|
||||
self.fieldname
|
||||
),
|
||||
'other_name': self.fieldname,
|
||||
}
|
||||
message = self.message
|
||||
|
|
|
@ -199,7 +199,7 @@ def nobr_f(v, c, m, p):
|
|||
def label_link(v, c, m, p):
|
||||
try:
|
||||
default_params = ast.literal_eval(m.default_params)
|
||||
except:
|
||||
except Exception:
|
||||
default_params = {}
|
||||
url = url_for(
|
||||
'airflow.chart', chart_id=m.id, iteration_no=m.iteration_no,
|
||||
|
@ -397,9 +397,9 @@ class Airflow(BaseView):
|
|||
# Processing templated fields
|
||||
try:
|
||||
args = ast.literal_eval(chart.default_params)
|
||||
if type(args) is not type(dict()):
|
||||
if not isinstance(args, dict):
|
||||
raise AirflowException('Not a dict')
|
||||
except:
|
||||
except Exception:
|
||||
args = {}
|
||||
payload['error'] += (
|
||||
"Default params is not valid, string has to evaluate as "
|
||||
|
@ -441,15 +441,15 @@ class Airflow(BaseView):
|
|||
if not payload['error'] and len(df) == 0:
|
||||
payload['error'] += "Empty result set. "
|
||||
elif (
|
||||
not payload['error'] and
|
||||
chart.sql_layout == 'series' and
|
||||
chart.chart_type != "datatable" and
|
||||
len(df.columns) < 3):
|
||||
not payload['error'] and
|
||||
chart.sql_layout == 'series' and
|
||||
chart.chart_type != "datatable" and
|
||||
len(df.columns) < 3):
|
||||
payload['error'] += "SQL needs to return at least 3 columns. "
|
||||
elif (
|
||||
not payload['error'] and
|
||||
chart.sql_layout == 'columns' and
|
||||
len(df.columns) < 2):
|
||||
not payload['error'] and
|
||||
chart.sql_layout == 'columns' and
|
||||
len(df.columns) < 2):
|
||||
payload['error'] += "SQL needs to return at least 2 columns. "
|
||||
elif not payload['error']:
|
||||
import numpy as np
|
||||
|
@ -615,13 +615,13 @@ class Airflow(BaseView):
|
|||
# If no dag_run is active, return task instances from most recent dag_run.
|
||||
LastTI = (
|
||||
session.query(TI.dag_id.label('dag_id'), TI.state.label('state'))
|
||||
.join(LastDagRun, and_(
|
||||
.join(LastDagRun, and_(
|
||||
LastDagRun.c.dag_id == TI.dag_id,
|
||||
LastDagRun.c.execution_date == TI.execution_date))
|
||||
)
|
||||
RunningTI = (
|
||||
session.query(TI.dag_id.label('dag_id'), TI.state.label('state'))
|
||||
.join(RunningDagRun, and_(
|
||||
.join(RunningDagRun, and_(
|
||||
RunningDagRun.c.dag_id == TI.dag_id,
|
||||
RunningDagRun.c.execution_date == TI.execution_date))
|
||||
)
|
||||
|
@ -629,7 +629,7 @@ class Airflow(BaseView):
|
|||
UnionTI = union_all(LastTI, RunningTI).alias('union_ti')
|
||||
qry = (
|
||||
session.query(UnionTI.c.dag_id, UnionTI.c.state, sqla.func.count())
|
||||
.group_by(UnionTI.c.dag_id, UnionTI.c.state)
|
||||
.group_by(UnionTI.c.dag_id, UnionTI.c.state)
|
||||
)
|
||||
|
||||
data = {}
|
||||
|
@ -645,7 +645,7 @@ class Airflow(BaseView):
|
|||
for state in State.task_states:
|
||||
try:
|
||||
count = data[dag.dag_id][state]
|
||||
except:
|
||||
except Exception:
|
||||
count = 0
|
||||
d = {
|
||||
'state': state,
|
||||
|
@ -1632,13 +1632,13 @@ class Airflow(BaseView):
|
|||
TF = models.TaskFail
|
||||
ti_fails = (
|
||||
session
|
||||
.query(TF)
|
||||
.filter(
|
||||
.query(TF)
|
||||
.filter(
|
||||
TF.dag_id == dag.dag_id,
|
||||
TF.execution_date >= min_date,
|
||||
TF.execution_date <= base_date,
|
||||
TF.task_id.in_([t.task_id for t in dag.tasks]))
|
||||
.all()
|
||||
.all()
|
||||
)
|
||||
|
||||
fails_totals = defaultdict(int)
|
||||
|
@ -1995,7 +1995,7 @@ class Airflow(BaseView):
|
|||
return self.render(
|
||||
'airflow/variables/{}.html'.format(form)
|
||||
)
|
||||
except:
|
||||
except Exception:
|
||||
# prevent XSS
|
||||
form = escape(form)
|
||||
return ("Error: form airflow/variables/{}.html "
|
||||
|
@ -2555,7 +2555,7 @@ class VariableView(wwwutils.DataProfilingMixin, AirflowModelView):
|
|||
val = None
|
||||
try:
|
||||
val = d.decode(var.val)
|
||||
except:
|
||||
except Exception:
|
||||
val = var.val
|
||||
var_dict[var.key] = val
|
||||
|
||||
|
@ -2961,7 +2961,7 @@ class ConnectionModelView(wwwutils.SuperUserMixin, AirflowModelView):
|
|||
fk = None
|
||||
try:
|
||||
fk = conf.get('core', 'fernet_key')
|
||||
except:
|
||||
except Exception:
|
||||
pass
|
||||
return fk is None
|
||||
|
||||
|
@ -2973,10 +2973,10 @@ class ConnectionModelView(wwwutils.SuperUserMixin, AirflowModelView):
|
|||
"""
|
||||
is_secure = False
|
||||
try:
|
||||
import cryptography
|
||||
import cryptography # noqa F401
|
||||
conf.get('core', 'fernet_key')
|
||||
is_secure = True
|
||||
except:
|
||||
except Exception:
|
||||
pass
|
||||
return is_secure
|
||||
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
|
|
Загрузка…
Ссылка в новой задаче