Skip to content

Commit

Permalink
[AIRFLOW-6171] Apply .airflowignore to correct subdirectories (apache…
Browse files Browse the repository at this point in the history
…#6784)

Fix the defect that applied .airflowignore rules from one subdirectory
to all other subdirectories scanned later.
  • Loading branch information
diggerk authored and ashb committed Dec 12, 2019
1 parent 699aea8 commit 702005f
Show file tree
Hide file tree
Showing 6 changed files with 63 additions and 10 deletions.
2 changes: 1 addition & 1 deletion airflow/utils/file.py
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ def list_py_file_paths(directory: str,
# We want patterns defined in a parent folder's .airflowignore to
# apply to subdirs too
for subdir in dirs:
patterns_by_dir[os.path.join(root, subdir)] = patterns
patterns_by_dir[os.path.join(root, subdir)] = patterns.copy()

find_dag_file_paths(file_paths, files, patterns, root, safe_mode)
if include_examples:
Expand Down
5 changes: 1 addition & 4 deletions pytest.ini
Original file line number Diff line number Diff line change
Expand Up @@ -21,16 +21,13 @@ addopts =
--verbosity=1
; This will treat all tests as flaky
; --force-flaky
# DAG files
--ignore=tests/dags/test_clear_subdag.py
--ignore=tests/dags/test_impersonation_subdag.py
--ignore=tests/dags/test_subdag.py
norecursedirs =
.eggs
airflow
tests/dags_with_system_exit
tests/test_utils
tests/dags_corrupted
tests/dags
faulthandler_timeout = 480
log_print = True
log_level = INFO
1 change: 1 addition & 0 deletions tests/dags/subdir1/.airflowignore
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
.*_ignore_this.py # Ignore files ending with "_ignore_this.py"
23 changes: 23 additions & 0 deletions tests/dags/subdir1/test_ignore_this.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
# -*- coding: utf-8 -*-
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# 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.

# needed to work against airflow "safe mode" parsing
from airflow.models import DAG # noqa # pylint: disable=unused-import

raise Exception("This dag file should have been ignored!")
30 changes: 30 additions & 0 deletions tests/dags/subdir2/test_dont_ignore_this.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
# -*- coding: utf-8 -*-
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# 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.
from datetime import datetime

from airflow.models import DAG
from airflow.operators.bash_operator import BashOperator

DEFAULT_DATE = datetime(2019, 12, 1)

dag = DAG(dag_id='test_dag_under_subdir2', start_date=DEFAULT_DATE, schedule_interval=None)
task = BashOperator(
task_id='task1',
bash_command='echo "test dag under sub directory subdir2"',
dag=dag)
12 changes: 7 additions & 5 deletions tests/jobs/test_scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -2642,12 +2642,14 @@ def test_list_py_file_paths(self):
'no_dags.py',
'test_invalid_cron.py',
'test_zip_invalid_cron.zip',
'test_ignore_this.py',
]
for file_name in os.listdir(TEST_DAG_FOLDER):
if file_name.endswith('.py') or file_name.endswith('.zip'):
if file_name not in ignored_files:
expected_files.add(
'{}/{}'.format(TEST_DAG_FOLDER, file_name))
for root, dirs, files in os.walk(TEST_DAG_FOLDER):
for file_name in files:
if file_name.endswith('.py') or file_name.endswith('.zip'):
if file_name not in ignored_files:
expected_files.add(
'{}/{}'.format(root, file_name))
for file_path in list_py_file_paths(TEST_DAG_FOLDER, include_examples=False):
detected_files.add(file_path)
self.assertEqual(detected_files, expected_files)
Expand Down

0 comments on commit 702005f

Please sign in to comment.